Re: Multi user setup and saving a DataFrame / RDD to a network exported file system

2015-05-21 Thread Tomasz Fruboes

Hi,

 thanks for answer, I'll open a ticket.

 In the meantime - I have found a workaround. The recipe is the following:

1. Create a new account/group on all machines (lets call it sparkuser). 
Run spark from this account.


2. Add your user to group sparkuser.

3. If you decide to write RDD/parquet file under "workdir" directory you 
need to execute the following (just once, before running spark-submit):


chgrp sparkuser workdir
chmod g+s workdir
setfacl -d -m g::rwx workdir

(first two steps can be replaced also by "newgrp sparkuser", but this 
way all your files will be created with sparkuser group)


 than calls like

rdd.saveAsPickleFile(workdir+"/somename")

 work just fine.

 The above solution has one serious problem - any other user from 
sparkuser group will be able to overwrite your saved data.


 cheers,
   Tomasz






W dniu 20.05.2015 o 23:08, Davies Liu pisze:

Could you file a JIRA for this?

The executor should run under the user who submit a job, I think.

On Wed, May 20, 2015 at 2:40 AM, Tomasz Fruboes
 wrote:

Thanks for a suggestion. I have tried playing with it, sc.sparkUser() gives
me expected user name, but it doesnt solve the problem. From a quick search
through the spark code it seems to me, that this setting is effective only
for yarn and mesos.

  I think the workaround for the problem could be using "--deploy-mode
cluster" (not 100% convenient, since disallows any interactive work), but
this is not supported for python based programs.

Cheers,
   Tomasz



W dniu 20.05.2015 o 10:57, Iulian Dragoș pisze:


You could try setting `SPARK_USER` to the user under which your workers
are running. I couldn't find many references to this variable, but at
least Yarn and Mesos take it into account when spawning executors.
Chances are that standalone mode also does it.

iulian

On Wed, May 20, 2015 at 9:29 AM, Tomasz Fruboes
mailto:tomasz.frub...@fuw.edu.pl>> wrote:

 Hi,

   thanks for answer. The rights are

 drwxr-xr-x 3 tfruboes all 5632 05-19 15 :40
 test19EE/

   I have tried setting the rights to 777 for this directory prior to
 execution. This does not get propagated down the chain, ie the
 directory created as a result of the "save" call
 (namesAndAges.parquet2 in the path in the dump [1] below) is created
 with the drwxr-xr-x rights (owned by the user submitting the job, ie
 tfruboes). The temp directories created inside

 namesAndAges.parquet2/_temporary/0/

 (e.g. task_201505200920_0009_r_01) are owned by root, again with
 drwxr-xr-x access rights

   Cheers,
Tomasz

 W dniu 19.05.2015 o 23:56, Davies Liu pisze:

 It surprises me, could you list the owner information of
 /mnt/lustre/bigdata/med_home/tmp/test19EE/ ?

 On Tue, May 19, 2015 at 8:15 AM, Tomasz Fruboes
 mailto:tomasz.frub...@fuw.edu.pl>>

 wrote:

 Dear Experts,

we have a spark cluster (standalone mode) in which master
 and workers are
 started from root account. Everything runs correctly to the
 point when we
 try doing operations such as

   dataFrame.select("name", "age").save(ofile, "parquet")

 or

   rdd.saveAsPickleFile(ofile)

 , where ofile is path on a network exported filesystem
 (visible on all
 nodes, in our case this is lustre, I guess on nfs effect
 would be similar).

Unsurprisingly temp files created on workers are owned by
 root, which then
 leads to a crash (see [1] below). Is there a
 solution/workaround for this
 (e.g. controlling file creation mode of the temporary files)?

 Cheers,
Tomasz


 ps I've tried to google this problem, couple of similar
 reports, but no
 clear answer/solution found

 ps2 For completeness - running master/workers as a regular
 user solves the
 problem only for the given user. For other users submitting
 to this master
 the result is given in [2] below


 [0] Cluster details:
 Master/workers: centos 6.5
 Spark 1.3.1 prebuilt for hadoop 2.4 (same behaviour for the
 2.6 build)


 [1]

##
  File

"/mnt/home/tfruboes/2015.05.SparkLocal/spark-1.3.1-bin-hadoop2.4/python/lib/py4j-0.8.2.1-src.zip/py4j/protocol.py",
 line 300, in get_return_value
 py4j.protocol.Py4JJavaError: An error occurred while calling
 o27.save.
 : java.io.IOException: Failed to rename

DeprecatedRawLocalFileStatus{path=file:/mnt/lustre/bigdata/med_home/tmp/test19EE/namesAndAges.parquet2/_temporary/0/task_201505191540_0009_r_01/part-r-2.parquet;

Re: Multi user setup and saving a DataFrame / RDD to a network exported file system

2015-05-20 Thread Davies Liu
Could you file a JIRA for this?

The executor should run under the user who submit a job, I think.

On Wed, May 20, 2015 at 2:40 AM, Tomasz Fruboes
 wrote:
> Thanks for a suggestion. I have tried playing with it, sc.sparkUser() gives
> me expected user name, but it doesnt solve the problem. From a quick search
> through the spark code it seems to me, that this setting is effective only
> for yarn and mesos.
>
>  I think the workaround for the problem could be using "--deploy-mode
> cluster" (not 100% convenient, since disallows any interactive work), but
> this is not supported for python based programs.
>
> Cheers,
>   Tomasz
>
>
>
> W dniu 20.05.2015 o 10:57, Iulian Dragoș pisze:
>>
>> You could try setting `SPARK_USER` to the user under which your workers
>> are running. I couldn't find many references to this variable, but at
>> least Yarn and Mesos take it into account when spawning executors.
>> Chances are that standalone mode also does it.
>>
>> iulian
>>
>> On Wed, May 20, 2015 at 9:29 AM, Tomasz Fruboes
>> mailto:tomasz.frub...@fuw.edu.pl>> wrote:
>>
>> Hi,
>>
>>   thanks for answer. The rights are
>>
>> drwxr-xr-x 3 tfruboes all 5632 05-19 15 :40
>> test19EE/
>>
>>   I have tried setting the rights to 777 for this directory prior to
>> execution. This does not get propagated down the chain, ie the
>> directory created as a result of the "save" call
>> (namesAndAges.parquet2 in the path in the dump [1] below) is created
>> with the drwxr-xr-x rights (owned by the user submitting the job, ie
>> tfruboes). The temp directories created inside
>>
>> namesAndAges.parquet2/_temporary/0/
>>
>> (e.g. task_201505200920_0009_r_01) are owned by root, again with
>> drwxr-xr-x access rights
>>
>>   Cheers,
>>Tomasz
>>
>> W dniu 19.05.2015 o 23:56, Davies Liu pisze:
>>
>> It surprises me, could you list the owner information of
>> /mnt/lustre/bigdata/med_home/tmp/test19EE/ ?
>>
>> On Tue, May 19, 2015 at 8:15 AM, Tomasz Fruboes
>> mailto:tomasz.frub...@fuw.edu.pl>>
>>
>> wrote:
>>
>> Dear Experts,
>>
>>we have a spark cluster (standalone mode) in which master
>> and workers are
>> started from root account. Everything runs correctly to the
>> point when we
>> try doing operations such as
>>
>>   dataFrame.select("name", "age").save(ofile, "parquet")
>>
>> or
>>
>>   rdd.saveAsPickleFile(ofile)
>>
>> , where ofile is path on a network exported filesystem
>> (visible on all
>> nodes, in our case this is lustre, I guess on nfs effect
>> would be similar).
>>
>>Unsurprisingly temp files created on workers are owned by
>> root, which then
>> leads to a crash (see [1] below). Is there a
>> solution/workaround for this
>> (e.g. controlling file creation mode of the temporary files)?
>>
>> Cheers,
>>Tomasz
>>
>>
>> ps I've tried to google this problem, couple of similar
>> reports, but no
>> clear answer/solution found
>>
>> ps2 For completeness - running master/workers as a regular
>> user solves the
>> problem only for the given user. For other users submitting
>> to this master
>> the result is given in [2] below
>>
>>
>> [0] Cluster details:
>> Master/workers: centos 6.5
>> Spark 1.3.1 prebuilt for hadoop 2.4 (same behaviour for the
>> 2.6 build)
>>
>>
>> [1]
>>
>> ##
>>  File
>>
>> "/mnt/home/tfruboes/2015.05.SparkLocal/spark-1.3.1-bin-hadoop2.4/python/lib/py4j-0.8.2.1-src.zip/py4j/protocol.py",
>> line 300, in get_return_value
>> py4j.protocol.Py4JJavaError: An error occurred while calling
>> o27.save.
>> : java.io.IOException: Failed to rename
>>
>> DeprecatedRawLocalFileStatus{path=file:/mnt/lustre/bigdata/med_home/tmp/test19EE/namesAndAges.parquet2/_temporary/0/task_201505191540_0009_r_01/part-r-2.parquet;
>> isDirectory=false; length=534; replication=1;
>> blocksize=33554432;
>> modification_time=1432042832000; access_time=0; owner=;
>> group=;
>> permission=rw-rw-rw-; isSymlink=false} to
>>
>> file:/mnt/lustre/bigdata/med_home/tmp/test19EE/namesAndAges.parquet2/part-r-2.parquet
>>   at
>>
>> org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.mergePaths(FileOutputCommitter.java:346)
>>   at
>>
>> org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.mergePaths(FileOutputCommitter.java:362)
>>   at
>>
>> org.apache.hadoop.mapreduce.lib

Re: Multi user setup and saving a DataFrame / RDD to a network exported file system

2015-05-20 Thread Tomasz Fruboes
Thanks for a suggestion. I have tried playing with it, sc.sparkUser() 
gives me expected user name, but it doesnt solve the problem. From a 
quick search through the spark code it seems to me, that this setting is 
effective only for yarn and mesos.


 I think the workaround for the problem could be using "--deploy-mode 
cluster" (not 100% convenient, since disallows any interactive work), 
but this is not supported for python based programs.


Cheers,
  Tomasz



W dniu 20.05.2015 o 10:57, Iulian Dragoș pisze:

You could try setting `SPARK_USER` to the user under which your workers
are running. I couldn't find many references to this variable, but at
least Yarn and Mesos take it into account when spawning executors.
Chances are that standalone mode also does it.

iulian

On Wed, May 20, 2015 at 9:29 AM, Tomasz Fruboes
mailto:tomasz.frub...@fuw.edu.pl>> wrote:

Hi,

  thanks for answer. The rights are

drwxr-xr-x 3 tfruboes all 5632 05-19 15 :40
test19EE/

  I have tried setting the rights to 777 for this directory prior to
execution. This does not get propagated down the chain, ie the
directory created as a result of the "save" call
(namesAndAges.parquet2 in the path in the dump [1] below) is created
with the drwxr-xr-x rights (owned by the user submitting the job, ie
tfruboes). The temp directories created inside

namesAndAges.parquet2/_temporary/0/

(e.g. task_201505200920_0009_r_01) are owned by root, again with
drwxr-xr-x access rights

  Cheers,
   Tomasz

W dniu 19.05.2015 o 23:56, Davies Liu pisze:

It surprises me, could you list the owner information of
/mnt/lustre/bigdata/med_home/tmp/test19EE/ ?

On Tue, May 19, 2015 at 8:15 AM, Tomasz Fruboes
mailto:tomasz.frub...@fuw.edu.pl>>
wrote:

Dear Experts,

   we have a spark cluster (standalone mode) in which master
and workers are
started from root account. Everything runs correctly to the
point when we
try doing operations such as

  dataFrame.select("name", "age").save(ofile, "parquet")

or

  rdd.saveAsPickleFile(ofile)

, where ofile is path on a network exported filesystem
(visible on all
nodes, in our case this is lustre, I guess on nfs effect
would be similar).

   Unsurprisingly temp files created on workers are owned by
root, which then
leads to a crash (see [1] below). Is there a
solution/workaround for this
(e.g. controlling file creation mode of the temporary files)?

Cheers,
   Tomasz


ps I've tried to google this problem, couple of similar
reports, but no
clear answer/solution found

ps2 For completeness - running master/workers as a regular
user solves the
problem only for the given user. For other users submitting
to this master
the result is given in [2] below


[0] Cluster details:
Master/workers: centos 6.5
Spark 1.3.1 prebuilt for hadoop 2.4 (same behaviour for the
2.6 build)


[1]
##
 File

"/mnt/home/tfruboes/2015.05.SparkLocal/spark-1.3.1-bin-hadoop2.4/python/lib/py4j-0.8.2.1-src.zip/py4j/protocol.py",
line 300, in get_return_value
py4j.protocol.Py4JJavaError: An error occurred while calling
o27.save.
: java.io.IOException: Failed to rename

DeprecatedRawLocalFileStatus{path=file:/mnt/lustre/bigdata/med_home/tmp/test19EE/namesAndAges.parquet2/_temporary/0/task_201505191540_0009_r_01/part-r-2.parquet;
isDirectory=false; length=534; replication=1;
blocksize=33554432;
modification_time=1432042832000; access_time=0; owner=; group=;
permission=rw-rw-rw-; isSymlink=false} to

file:/mnt/lustre/bigdata/med_home/tmp/test19EE/namesAndAges.parquet2/part-r-2.parquet
  at

org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.mergePaths(FileOutputCommitter.java:346)
  at

org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.mergePaths(FileOutputCommitter.java:362)
  at

org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.commitJob(FileOutputCommitter.java:310)
  at

parquet.hadoop.ParquetOutputCommitter.commitJob(ParquetOutputCommitter.java:43)
  at

org.apache.spark.sql.parquet.ParquetRelation2.insert(newParquet.scala:690)
  at

org.apache.spark.sql.parquet.DefaultSource.createRelation(newParquet.s

Re: Multi user setup and saving a DataFrame / RDD to a network exported file system

2015-05-20 Thread Iulian Dragoș
You could try setting `SPARK_USER` to the user under which your workers are
running. I couldn't find many references to this variable, but at least
Yarn and Mesos take it into account when spawning executors. Chances are
that standalone mode also does it.

iulian

On Wed, May 20, 2015 at 9:29 AM, Tomasz Fruboes 
wrote:

> Hi,
>
>  thanks for answer. The rights are
>
> drwxr-xr-x 3 tfruboes all 5632 05-19 15:40 test19EE/
>
>  I have tried setting the rights to 777 for this directory prior to
> execution. This does not get propagated down the chain, ie the directory
> created as a result of the "save" call (namesAndAges.parquet2 in the path
> in the dump [1] below) is created with the drwxr-xr-x rights (owned by the
> user submitting the job, ie tfruboes). The temp directories created inside
>
> namesAndAges.parquet2/_temporary/0/
>
> (e.g. task_201505200920_0009_r_01) are owned by root, again with
> drwxr-xr-x access rights
>
>  Cheers,
>   Tomasz
>
> W dniu 19.05.2015 o 23:56, Davies Liu pisze:
>
>  It surprises me, could you list the owner information of
>> /mnt/lustre/bigdata/med_home/tmp/test19EE/ ?
>>
>> On Tue, May 19, 2015 at 8:15 AM, Tomasz Fruboes
>>  wrote:
>>
>>> Dear Experts,
>>>
>>>   we have a spark cluster (standalone mode) in which master and workers
>>> are
>>> started from root account. Everything runs correctly to the point when we
>>> try doing operations such as
>>>
>>>  dataFrame.select("name", "age").save(ofile, "parquet")
>>>
>>> or
>>>
>>>  rdd.saveAsPickleFile(ofile)
>>>
>>> , where ofile is path on a network exported filesystem (visible on all
>>> nodes, in our case this is lustre, I guess on nfs effect would be
>>> similar).
>>>
>>>   Unsurprisingly temp files created on workers are owned by root, which
>>> then
>>> leads to a crash (see [1] below). Is there a solution/workaround for this
>>> (e.g. controlling file creation mode of the temporary files)?
>>>
>>> Cheers,
>>>   Tomasz
>>>
>>>
>>> ps I've tried to google this problem, couple of similar reports, but no
>>> clear answer/solution found
>>>
>>> ps2 For completeness - running master/workers as a regular user solves
>>> the
>>> problem only for the given user. For other users submitting to this
>>> master
>>> the result is given in [2] below
>>>
>>>
>>> [0] Cluster details:
>>> Master/workers: centos 6.5
>>> Spark 1.3.1 prebuilt for hadoop 2.4 (same behaviour for the 2.6 build)
>>>
>>>
>>> [1]
>>> ##
>>> File
>>>
>>> "/mnt/home/tfruboes/2015.05.SparkLocal/spark-1.3.1-bin-hadoop2.4/python/lib/py4j-0.8.2.1-src.zip/py4j/protocol.py",
>>> line 300, in get_return_value
>>> py4j.protocol.Py4JJavaError: An error occurred while calling o27.save.
>>> : java.io.IOException: Failed to rename
>>>
>>> DeprecatedRawLocalFileStatus{path=file:/mnt/lustre/bigdata/med_home/tmp/test19EE/namesAndAges.parquet2/_temporary/0/task_201505191540_0009_r_01/part-r-2.parquet;
>>> isDirectory=false; length=534; replication=1; blocksize=33554432;
>>> modification_time=1432042832000; access_time=0; owner=; group=;
>>> permission=rw-rw-rw-; isSymlink=false} to
>>>
>>> file:/mnt/lustre/bigdata/med_home/tmp/test19EE/namesAndAges.parquet2/part-r-2.parquet
>>>  at
>>>
>>> org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.mergePaths(FileOutputCommitter.java:346)
>>>  at
>>>
>>> org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.mergePaths(FileOutputCommitter.java:362)
>>>  at
>>>
>>> org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.commitJob(FileOutputCommitter.java:310)
>>>  at
>>>
>>> parquet.hadoop.ParquetOutputCommitter.commitJob(ParquetOutputCommitter.java:43)
>>>  at
>>>
>>> org.apache.spark.sql.parquet.ParquetRelation2.insert(newParquet.scala:690)
>>>  at
>>>
>>> org.apache.spark.sql.parquet.DefaultSource.createRelation(newParquet.scala:129)
>>>  at
>>> org.apache.spark.sql.sources.ResolvedDataSource$.apply(ddl.scala:240)
>>>  at org.apache.spark.sql.DataFrame.save(DataFrame.scala:1196)
>>>  at org.apache.spark.sql.DataFrame.save(DataFrame.scala:1181)
>>>  at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>>>  at
>>>
>>> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
>>>  at
>>>
>>> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>>>  at java.lang.reflect.Method.invoke(Method.java:606)
>>>  at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:231)
>>>  at
>>> py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:379)
>>>  at py4j.Gateway.invoke(Gateway.java:259)
>>>  at
>>> py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:133)
>>>  at py4j.commands.CallCommand.execute(CallCommand.java:79)
>>>  at py4j.GatewayConnection.run(GatewayConnection.java:207)
>>>  at java.lang.Th

Re: Multi user setup and saving a DataFrame / RDD to a network exported file system

2015-05-20 Thread Tomasz Fruboes

Hi,

 thanks for answer. The rights are

drwxr-xr-x 3 tfruboes all 5632 05-19 15:40 test19EE/

 I have tried setting the rights to 777 for this directory prior to 
execution. This does not get propagated down the chain, ie the directory 
created as a result of the "save" call (namesAndAges.parquet2 in the 
path in the dump [1] below) is created with the drwxr-xr-x rights (owned 
by the user submitting the job, ie tfruboes). The temp directories 
created inside


namesAndAges.parquet2/_temporary/0/

(e.g. task_201505200920_0009_r_01) are owned by root, again with 
drwxr-xr-x access rights


 Cheers,
  Tomasz

W dniu 19.05.2015 o 23:56, Davies Liu pisze:

It surprises me, could you list the owner information of
/mnt/lustre/bigdata/med_home/tmp/test19EE/ ?

On Tue, May 19, 2015 at 8:15 AM, Tomasz Fruboes
 wrote:

Dear Experts,

  we have a spark cluster (standalone mode) in which master and workers are
started from root account. Everything runs correctly to the point when we
try doing operations such as

 dataFrame.select("name", "age").save(ofile, "parquet")

or

 rdd.saveAsPickleFile(ofile)

, where ofile is path on a network exported filesystem (visible on all
nodes, in our case this is lustre, I guess on nfs effect would be similar).

  Unsurprisingly temp files created on workers are owned by root, which then
leads to a crash (see [1] below). Is there a solution/workaround for this
(e.g. controlling file creation mode of the temporary files)?

Cheers,
  Tomasz


ps I've tried to google this problem, couple of similar reports, but no
clear answer/solution found

ps2 For completeness - running master/workers as a regular user solves the
problem only for the given user. For other users submitting to this master
the result is given in [2] below


[0] Cluster details:
Master/workers: centos 6.5
Spark 1.3.1 prebuilt for hadoop 2.4 (same behaviour for the 2.6 build)


[1]
##
File
"/mnt/home/tfruboes/2015.05.SparkLocal/spark-1.3.1-bin-hadoop2.4/python/lib/py4j-0.8.2.1-src.zip/py4j/protocol.py",
line 300, in get_return_value
py4j.protocol.Py4JJavaError: An error occurred while calling o27.save.
: java.io.IOException: Failed to rename
DeprecatedRawLocalFileStatus{path=file:/mnt/lustre/bigdata/med_home/tmp/test19EE/namesAndAges.parquet2/_temporary/0/task_201505191540_0009_r_01/part-r-2.parquet;
isDirectory=false; length=534; replication=1; blocksize=33554432;
modification_time=1432042832000; access_time=0; owner=; group=;
permission=rw-rw-rw-; isSymlink=false} to
file:/mnt/lustre/bigdata/med_home/tmp/test19EE/namesAndAges.parquet2/part-r-2.parquet
 at
org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.mergePaths(FileOutputCommitter.java:346)
 at
org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.mergePaths(FileOutputCommitter.java:362)
 at
org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.commitJob(FileOutputCommitter.java:310)
 at
parquet.hadoop.ParquetOutputCommitter.commitJob(ParquetOutputCommitter.java:43)
 at
org.apache.spark.sql.parquet.ParquetRelation2.insert(newParquet.scala:690)
 at
org.apache.spark.sql.parquet.DefaultSource.createRelation(newParquet.scala:129)
 at
org.apache.spark.sql.sources.ResolvedDataSource$.apply(ddl.scala:240)
 at org.apache.spark.sql.DataFrame.save(DataFrame.scala:1196)
 at org.apache.spark.sql.DataFrame.save(DataFrame.scala:1181)
 at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
 at
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
 at
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
 at java.lang.reflect.Method.invoke(Method.java:606)
 at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:231)
 at
py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:379)
 at py4j.Gateway.invoke(Gateway.java:259)
 at
py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:133)
 at py4j.commands.CallCommand.execute(CallCommand.java:79)
 at py4j.GatewayConnection.run(GatewayConnection.java:207)
 at java.lang.Thread.run(Thread.java:745)
##



[2]
##
15/05/19 14:45:19 WARN TaskSetManager: Lost task 0.0 in stage 2.0 (TID 3,
wn23023.cis.gov.pl): java.io.IOException: Mkdirs failed to create
file:/mnt/lustre/bigdata/med_home/tmp/test18/namesAndAges.parquet2/_temporary/0/_temporary/attempt_201505191445_0009_r_00_0
 at
org.apache.hadoop.fs.ChecksumFileSystem.create(ChecksumFileSystem.java:438)
 at
org.apache.hadoop.fs.ChecksumFileSystem.create(ChecksumFileSystem.java:424)
 at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:906)
 at org.apache.hadoop.fs.FileSystem.crea

Re: Multi user setup and saving a DataFrame / RDD to a network exported file system

2015-05-19 Thread Davies Liu
It surprises me, could you list the owner information of
/mnt/lustre/bigdata/med_home/tmp/test19EE/ ?

On Tue, May 19, 2015 at 8:15 AM, Tomasz Fruboes
 wrote:
> Dear Experts,
>
>  we have a spark cluster (standalone mode) in which master and workers are
> started from root account. Everything runs correctly to the point when we
> try doing operations such as
>
> dataFrame.select("name", "age").save(ofile, "parquet")
>
> or
>
> rdd.saveAsPickleFile(ofile)
>
> , where ofile is path on a network exported filesystem (visible on all
> nodes, in our case this is lustre, I guess on nfs effect would be similar).
>
>  Unsurprisingly temp files created on workers are owned by root, which then
> leads to a crash (see [1] below). Is there a solution/workaround for this
> (e.g. controlling file creation mode of the temporary files)?
>
> Cheers,
>  Tomasz
>
>
> ps I've tried to google this problem, couple of similar reports, but no
> clear answer/solution found
>
> ps2 For completeness - running master/workers as a regular user solves the
> problem only for the given user. For other users submitting to this master
> the result is given in [2] below
>
>
> [0] Cluster details:
> Master/workers: centos 6.5
> Spark 1.3.1 prebuilt for hadoop 2.4 (same behaviour for the 2.6 build)
>
>
> [1]
> ##
>File
> "/mnt/home/tfruboes/2015.05.SparkLocal/spark-1.3.1-bin-hadoop2.4/python/lib/py4j-0.8.2.1-src.zip/py4j/protocol.py",
> line 300, in get_return_value
> py4j.protocol.Py4JJavaError: An error occurred while calling o27.save.
> : java.io.IOException: Failed to rename
> DeprecatedRawLocalFileStatus{path=file:/mnt/lustre/bigdata/med_home/tmp/test19EE/namesAndAges.parquet2/_temporary/0/task_201505191540_0009_r_01/part-r-2.parquet;
> isDirectory=false; length=534; replication=1; blocksize=33554432;
> modification_time=1432042832000; access_time=0; owner=; group=;
> permission=rw-rw-rw-; isSymlink=false} to
> file:/mnt/lustre/bigdata/med_home/tmp/test19EE/namesAndAges.parquet2/part-r-2.parquet
> at
> org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.mergePaths(FileOutputCommitter.java:346)
> at
> org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.mergePaths(FileOutputCommitter.java:362)
> at
> org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.commitJob(FileOutputCommitter.java:310)
> at
> parquet.hadoop.ParquetOutputCommitter.commitJob(ParquetOutputCommitter.java:43)
> at
> org.apache.spark.sql.parquet.ParquetRelation2.insert(newParquet.scala:690)
> at
> org.apache.spark.sql.parquet.DefaultSource.createRelation(newParquet.scala:129)
> at
> org.apache.spark.sql.sources.ResolvedDataSource$.apply(ddl.scala:240)
> at org.apache.spark.sql.DataFrame.save(DataFrame.scala:1196)
> at org.apache.spark.sql.DataFrame.save(DataFrame.scala:1181)
> at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> at
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
> at
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> at java.lang.reflect.Method.invoke(Method.java:606)
> at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:231)
> at
> py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:379)
> at py4j.Gateway.invoke(Gateway.java:259)
> at
> py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:133)
> at py4j.commands.CallCommand.execute(CallCommand.java:79)
> at py4j.GatewayConnection.run(GatewayConnection.java:207)
> at java.lang.Thread.run(Thread.java:745)
> ##
>
>
>
> [2]
> ##
> 15/05/19 14:45:19 WARN TaskSetManager: Lost task 0.0 in stage 2.0 (TID 3,
> wn23023.cis.gov.pl): java.io.IOException: Mkdirs failed to create
> file:/mnt/lustre/bigdata/med_home/tmp/test18/namesAndAges.parquet2/_temporary/0/_temporary/attempt_201505191445_0009_r_00_0
> at
> org.apache.hadoop.fs.ChecksumFileSystem.create(ChecksumFileSystem.java:438)
> at
> org.apache.hadoop.fs.ChecksumFileSystem.create(ChecksumFileSystem.java:424)
> at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:906)
> at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:887)
> at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:784)
> at
> parquet.hadoop.ParquetFileWriter.(ParquetFileWriter.java:154)
> at
> parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:279)
> at
> parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:252)
> at
> org.apache.spark.sql.parquet.ParquetRelation2.org$apache$spark$sql$parquet$ParquetRelation2$$writeShard$1(newParquet.scala:667)
> at
> org

Multi user setup and saving a DataFrame / RDD to a network exported file system

2015-05-19 Thread Tomasz Fruboes

Dear Experts,

 we have a spark cluster (standalone mode) in which master and workers 
are started from root account. Everything runs correctly to the point 
when we try doing operations such as


dataFrame.select("name", "age").save(ofile, "parquet")

or

rdd.saveAsPickleFile(ofile)

, where ofile is path on a network exported filesystem (visible on all 
nodes, in our case this is lustre, I guess on nfs effect would be similar).


 Unsurprisingly temp files created on workers are owned by root, which 
then leads to a crash (see [1] below). Is there a solution/workaround 
for this (e.g. controlling file creation mode of the temporary files)?


Cheers,
 Tomasz


ps I've tried to google this problem, couple of similar reports, but no 
clear answer/solution found


ps2 For completeness - running master/workers as a regular user solves 
the problem only for the given user. For other users submitting to this 
master the result is given in [2] below



[0] Cluster details:
Master/workers: centos 6.5
Spark 1.3.1 prebuilt for hadoop 2.4 (same behaviour for the 2.6 build)


[1]
##
   File 
"/mnt/home/tfruboes/2015.05.SparkLocal/spark-1.3.1-bin-hadoop2.4/python/lib/py4j-0.8.2.1-src.zip/py4j/protocol.py", 
line 300, in get_return_value

py4j.protocol.Py4JJavaError: An error occurred while calling o27.save.
: java.io.IOException: Failed to rename 
DeprecatedRawLocalFileStatus{path=file:/mnt/lustre/bigdata/med_home/tmp/test19EE/namesAndAges.parquet2/_temporary/0/task_201505191540_0009_r_01/part-r-2.parquet; 
isDirectory=false; length=534; replication=1; blocksize=33554432; 
modification_time=1432042832000; access_time=0; owner=; group=; 
permission=rw-rw-rw-; isSymlink=false} to 
file:/mnt/lustre/bigdata/med_home/tmp/test19EE/namesAndAges.parquet2/part-r-2.parquet
at 
org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.mergePaths(FileOutputCommitter.java:346)
at 
org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.mergePaths(FileOutputCommitter.java:362)
at 
org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.commitJob(FileOutputCommitter.java:310)
at 
parquet.hadoop.ParquetOutputCommitter.commitJob(ParquetOutputCommitter.java:43)
at 
org.apache.spark.sql.parquet.ParquetRelation2.insert(newParquet.scala:690)
at 
org.apache.spark.sql.parquet.DefaultSource.createRelation(newParquet.scala:129)
at 
org.apache.spark.sql.sources.ResolvedDataSource$.apply(ddl.scala:240)

at org.apache.spark.sql.DataFrame.save(DataFrame.scala:1196)
at org.apache.spark.sql.DataFrame.save(DataFrame.scala:1181)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)

at java.lang.reflect.Method.invoke(Method.java:606)
at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:231)
at 
py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:379)

at py4j.Gateway.invoke(Gateway.java:259)
at 
py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:133)

at py4j.commands.CallCommand.execute(CallCommand.java:79)
at py4j.GatewayConnection.run(GatewayConnection.java:207)
at java.lang.Thread.run(Thread.java:745)
##



[2]
##
15/05/19 14:45:19 WARN TaskSetManager: Lost task 0.0 in stage 2.0 (TID 
3, wn23023.cis.gov.pl): java.io.IOException: Mkdirs failed to create 
file:/mnt/lustre/bigdata/med_home/tmp/test18/namesAndAges.parquet2/_temporary/0/_temporary/attempt_201505191445_0009_r_00_0
	at 
org.apache.hadoop.fs.ChecksumFileSystem.create(ChecksumFileSystem.java:438)
	at 
org.apache.hadoop.fs.ChecksumFileSystem.create(ChecksumFileSystem.java:424)

at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:906)
at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:887)
at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:784)
at parquet.hadoop.ParquetFileWriter.(ParquetFileWriter.java:154)
	at 
parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:279)
	at 
parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:252)
	at 
org.apache.spark.sql.parquet.ParquetRelation2.org$apache$spark$sql$parquet$ParquetRelation2$$writeShard$1(newParquet.scala:667)
	at 
org.apache.spark.sql.parquet.ParquetRelation2$$anonfun$insert$2.apply(newParquet.scala:689)
	at 
org.apache.spark.sql.parquet.ParquetRelation2$$anonfun$insert$2.apply(newParquet.scala:689)

at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:61)
at org.apache.spark.scheduler.Task.run(Task.scala:64)
at org.apache