RE: hdfs-ha on mesos - odd bug

2015-11-11 Thread Buttler, David

I have verified that this error exists on my system as well, and the suggested 
workaround also works.
Spark version: 1.5.1; 1.5.2
Mesos version: 0.21.1
CDH version: 4.7

I have set up the spark-env.sh to contain HADOOP_CONF_DIR pointing to the 
correct place, and I have also linked in the hdfs-site.xml file to 
$SPARK_HOME/conf.  I agree that it should work, but it doesn't.

I have also tried including the correct Hadoop configuration files in the 
application jar.

Note: it works fine from spark-shell, but it doesn't work from spark-submit

Dave

-Original Message-
From: Marcelo Vanzin [mailto:van...@cloudera.com] 
Sent: Tuesday, September 15, 2015 7:47 PM
To: Adrian Bridgett
Cc: user
Subject: Re: hdfs-ha on mesos - odd bug

On Mon, Sep 14, 2015 at 6:55 AM, Adrian Bridgett <adr...@opensignal.com> wrote:
> 15/09/14 13:00:25 WARN TaskSetManager: Lost task 0.0 in stage 0.0 (TID 
> 0,
> 10.1.200.245): java.lang.IllegalArgumentException:
> java.net.UnknownHostException: nameservice1
> at
> org.apache.hadoop.security.SecurityUtil.buildTokenService(SecurityUtil.java:377)
> at
> org.apache.hadoop.hdfs.NameNodeProxies.createNonHAProxy(NameNodeProxie
> s.java:310)

This looks like you're trying to connect to an HA HDFS service but you have not 
provided the proper hdfs-site.xml for your app; then, instead of recognizing 
"nameservice1" as an HA nameservice, it thinks it's an actual NN address, tries 
to connect to it, and fails.

If you provide the correct hdfs-site.xml to your app (by placing it in 
$SPARK_HOME/conf or setting HADOOP_CONF_DIR to point to the conf directory), it 
should work.

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



Re: hdfs-ha on mesos - odd bug

2015-09-15 Thread Marcelo Vanzin
On Mon, Sep 14, 2015 at 6:55 AM, Adrian Bridgett  wrote:
> 15/09/14 13:00:25 WARN TaskSetManager: Lost task 0.0 in stage 0.0 (TID 0,
> 10.1.200.245): java.lang.IllegalArgumentException:
> java.net.UnknownHostException: nameservice1
> at
> org.apache.hadoop.security.SecurityUtil.buildTokenService(SecurityUtil.java:377)
> at
> org.apache.hadoop.hdfs.NameNodeProxies.createNonHAProxy(NameNodeProxies.java:310)

This looks like you're trying to connect to an HA HDFS service but you
have not provided the proper hdfs-site.xml for your app; then, instead
of recognizing "nameservice1" as an HA nameservice, it thinks it's an
actual NN address, tries to connect to it, and fails.

If you provide the correct hdfs-site.xml to your app (by placing it in
$SPARK_HOME/conf or setting HADOOP_CONF_DIR to point to the conf
directory), it should work.

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



Re: hdfs-ha on mesos - odd bug

2015-09-15 Thread Adrian Bridgett
Hi Sam, in short, no, it's a traditional install as we plan to use spot 
instances and didn't want price spikes to kill off HDFS.


We're actually doing a bit of a hybrid, using spot instances for the 
mesos slaves, ondemand for the mesos masters.  So for the time being, 
putting hdfs on the masters (we'll probably move to multiple slave 
instance types to avoid losing too many when spot price spikes, but for 
now this is acceptable).   Masters running CDH5.


Using hdfs://current-hdfs-master:8020 works fine, however using 
hdfs://nameservice1 fails in the rather odd way described (well, more 
that the workaround actually works!)  I think there's some underlying 
bug here that's being exposed.



On 14/09/2015 22:27, Sam Bessalah wrote:
I don't know about the broken url. But are you running HDFS as a mesos 
framework? If so is it using mesos-dns?
Then you should resolve the namenode via hdfs:/// 



On Mon, Sep 14, 2015 at 3:55 PM, Adrian Bridgett 
> wrote:


I'm hitting an odd issue with running spark on mesos together with
HA-HDFS, with an even odder workaround.

In particular I get an error that it can't find the HDFS
nameservice unless I put in a _broken_ url (discovered that
workaround by mistake!).  core-site.xml, hdfs-site.xml is
distributed to the slave node - and that file is read since I
deliberately break the file then I get an error as you'd expect.

NB: This is a bit different to

http://mail-archives.us.apache.org/mod_mbox/spark-user/201402.mbox/%3c1392442185079-1549.p...@n3.nabble.com%3E


Spark 1.5.0:

t=sc.textFile("hdfs://nameservice1/tmp/issue")
t.count()
(fails)

t=sc.textFile("file://etc/passwd")
t.count()
(errors about bad url - should have an extra / of course)
t=sc.textFile("hdfs://nameservice1/tmp/issue")
t.count()
then it works!!!

I should say that using file:///etc/passwd or hdfs:///tmp/issue
both fail as well.  Unless preceded by a broken url.I've tried
setting spark.hadoop.cloneConf to true, no change.

Sample (broken) run:
15/09/14 13:00:14 DEBUG HadoopRDD: Creating new JobConf and
caching it for later re-use
15/09/14 13:00:14 DEBUG : address: ip-10-1-200-165/10.1.200.165
 isLoopbackAddress: false, with host
10.1.200.165 ip-10-1-200-165
15/09/14 13:00:14 DEBUG BlockReaderLocal:
dfs.client.use.legacy.blockreader.local = false
15/09/14 13:00:14 DEBUG BlockReaderLocal:
dfs.client.read.shortcircuit = false
15/09/14 13:00:14 DEBUG BlockReaderLocal:
dfs.client.domain.socket.data.traffic = false
15/09/14 13:00:14 DEBUG BlockReaderLocal: dfs.domain.socket.path =
/var/run/hdfs-sockets/dn
15/09/14 13:00:14 DEBUG HAUtil: No HA service delegation token
found for logical URI hdfs://nameservice1
15/09/14 13:00:14 DEBUG BlockReaderLocal:
dfs.client.use.legacy.blockreader.local = false
15/09/14 13:00:14 DEBUG BlockReaderLocal:
dfs.client.read.shortcircuit = false
15/09/14 13:00:14 DEBUG BlockReaderLocal:
dfs.client.domain.socket.data.traffic = false
15/09/14 13:00:14 DEBUG BlockReaderLocal: dfs.domain.socket.path =
/var/run/hdfs-sockets/dn
15/09/14 13:00:14 DEBUG RetryUtils: multipleLinearRandomRetry = null
15/09/14 13:00:14 DEBUG Server: rpcKind=RPC_PROTOCOL_BUFFER,
rpcRequestWrapperClass=class
org.apache.hadoop.ipc.ProtobufRpcEngine$RpcRequestWrapper,

rpcInvoker=org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker@6245f50b
15/09/14 13:00:14 DEBUG Client: getting client out of cache:
org.apache.hadoop.ipc.Client@267f0fd3
15/09/14 13:00:14 DEBUG NativeCodeLoader: Trying to load the
custom-built native-hadoop library...
15/09/14 13:00:14 DEBUG NativeCodeLoader: Loaded the native-hadoop
library
...
15/09/14 13:00:14 DEBUG Client: Connecting to
mesos-1.example.com/10.1.200.165:8020

15/09/14 13:00:14 DEBUG Client: IPC Client (1739425103) connection
to mesos-1.example.com/10.1.200.165:8020
 from ubuntu:
starting, having connections 1
15/09/14 13:00:14 DEBUG Client: IPC Client (1739425103) connection
to mesos-1.example.com/10.1.200.165:8020
 from ubuntu sending #0
15/09/14 13:00:14 DEBUG Client: IPC Client (1739425103) connection
to mesos-1.example.com/10.1.200.165:8020
 from ubuntu got
value #0
15/09/14 13:00:14 DEBUG ProtobufRpcEngine: Call: getFileInfo took 36ms
15/09/14 13:00:14 DEBUG FileInputFormat: Time taken to get
FileStatuses: 69
15/09/14 13:00:14 INFO FileInputFormat: Total input paths to
process : 1
15/09/14 13:00:14 DEBUG Client: IPC Client (1739425103) connection
to 

Re: hdfs-ha on mesos - odd bug

2015-09-15 Thread Steve Loughran

> On 15 Sep 2015, at 08:55, Adrian Bridgett  wrote:
> 
> Hi Sam, in short, no, it's a traditional install as we plan to use spot 
> instances and didn't want price spikes to kill off HDFS.
> 
> We're actually doing a bit of a hybrid, using spot instances for the mesos 
> slaves, ondemand for the mesos masters.  So for the time being, putting hdfs 
> on the masters (we'll probably move to multiple slave instance types to avoid 
> losing too many when spot price spikes, but for now this is acceptable).   
> Masters running CDH5.

It's incredibly dangerous using hdfs NNs on spot vms; a significant enough 
spike will lose all of them in one go, and there goes your entire filesystem. 
Have a static VM, maybe even backed by EBS.

If you look at Hadoop architectures from Hortonworks, Cloudera and Amazon 
themselves, the usual stance is HDFS on static nodes, spot instances for 
compute only

> 
> Using hdfs://current-hdfs-master:8020 works fine, however using 
> hdfs://nameservice1 fails in the rather odd way described (well, more that 
> the workaround actually works!)  I think there's some underlying bug here 
> that's being exposed. 


this sounds an issue orthogonal to spot instances. Maybe related to how JVMs 
cache DNS entries forever?

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



Re: hdfs-ha on mesos - odd bug

2015-09-15 Thread Iulian DragoČ™
I've seen similar traces, but couldn't track down the failure completely.
You are using Kerberos for your HDFS cluster, right? AFAIK Kerberos isn't
supported in Mesos deployments.

Can you resolve that host name (nameservice1) from the driver machine (ping
nameservice1)? Can it be resolved from the other machines in the cluster?

Does it help if you read using `newAPIHadoopFile` instead of `textFile`?

On Mon, Sep 14, 2015 at 3:55 PM, Adrian Bridgett 
wrote:

> I'm hitting an odd issue with running spark on mesos together with
> HA-HDFS, with an even odder workaround.
>
> In particular I get an error that it can't find the HDFS nameservice
> unless I put in a _broken_ url (discovered that workaround by mistake!).
> core-site.xml, hdfs-site.xml is distributed to the slave node - and that
> file is read since I deliberately break the file then I get an error as
> you'd expect.
>
> NB: This is a bit different to
> http://mail-archives.us.apache.org/mod_mbox/spark-user/201402.mbox/%3c1392442185079-1549.p...@n3.nabble.com%3E
>
>
> Spark 1.5.0:
>
> t=sc.textFile("hdfs://nameservice1/tmp/issue")
> t.count()
> (fails)
>
> t=sc.textFile("file://etc/passwd")
> t.count()
> (errors about bad url - should have an extra / of course)
> t=sc.textFile("hdfs://nameservice1/tmp/issue")
> t.count()
> then it works!!!
>
> I should say that using file:///etc/passwd or hdfs:///tmp/issue both fail
> as well.  Unless preceded by a broken url.I've tried setting
> spark.hadoop.cloneConf to true, no change.
>
> Sample (broken) run:
> 15/09/14 13:00:14 DEBUG HadoopRDD: Creating new JobConf and caching it for
> later re-use
> 15/09/14 13:00:14 DEBUG : address: ip-10-1-200-165/10.1.200.165
> isLoopbackAddress: false, with host 10.1.200.165 ip-10-1-200-165
> 15/09/14 13:00:14 DEBUG BlockReaderLocal:
> dfs.client.use.legacy.blockreader.local = false
> 15/09/14 13:00:14 DEBUG BlockReaderLocal: dfs.client.read.shortcircuit =
> false
> 15/09/14 13:00:14 DEBUG BlockReaderLocal:
> dfs.client.domain.socket.data.traffic = false
> 15/09/14 13:00:14 DEBUG BlockReaderLocal: dfs.domain.socket.path =
> /var/run/hdfs-sockets/dn
> 15/09/14 13:00:14 DEBUG HAUtil: No HA service delegation token found for
> logical URI hdfs://nameservice1
> 15/09/14 13:00:14 DEBUG BlockReaderLocal:
> dfs.client.use.legacy.blockreader.local = false
> 15/09/14 13:00:14 DEBUG BlockReaderLocal: dfs.client.read.shortcircuit =
> false
> 15/09/14 13:00:14 DEBUG BlockReaderLocal:
> dfs.client.domain.socket.data.traffic = false
> 15/09/14 13:00:14 DEBUG BlockReaderLocal: dfs.domain.socket.path =
> /var/run/hdfs-sockets/dn
> 15/09/14 13:00:14 DEBUG RetryUtils: multipleLinearRandomRetry = null
> 15/09/14 13:00:14 DEBUG Server: rpcKind=RPC_PROTOCOL_BUFFER,
> rpcRequestWrapperClass=class
> org.apache.hadoop.ipc.ProtobufRpcEngine$RpcRequestWrapper,
> rpcInvoker=org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker@6245f50b
> 15/09/14 13:00:14 DEBUG Client: getting client out of cache:
> org.apache.hadoop.ipc.Client@267f0fd3
> 15/09/14 13:00:14 DEBUG NativeCodeLoader: Trying to load the custom-built
> native-hadoop library...
> 15/09/14 13:00:14 DEBUG NativeCodeLoader: Loaded the native-hadoop library
> ...
> 15/09/14 13:00:14 DEBUG Client: Connecting to
> mesos-1.example.com/10.1.200.165:8020
> 15/09/14 13:00:14 DEBUG Client: IPC Client (1739425103) connection to
> mesos-1.example.com/10.1.200.165:8020 from ubuntu: starting, having
> connections 1
> 15/09/14 13:00:14 DEBUG Client: IPC Client (1739425103) connection to
> mesos-1.example.com/10.1.200.165:8020 from ubuntu sending #0
> 15/09/14 13:00:14 DEBUG Client: IPC Client (1739425103) connection to
> mesos-1.example.com/10.1.200.165:8020 from ubuntu got value #0
> 15/09/14 13:00:14 DEBUG ProtobufRpcEngine: Call: getFileInfo took 36ms
> 15/09/14 13:00:14 DEBUG FileInputFormat: Time taken to get FileStatuses: 69
> 15/09/14 13:00:14 INFO FileInputFormat: Total input paths to process : 1
> 15/09/14 13:00:14 DEBUG Client: IPC Client (1739425103) connection to
> mesos-1.example.com/10.1.200.165:8020 from ubuntu sending #1
> 15/09/14 13:00:14 DEBUG Client: IPC Client (1739425103) connection to
> mesos-1.example.com/10.1.200.165:8020 from ubuntu got value #1
> 15/09/14 13:00:14 DEBUG ProtobufRpcEngine: Call: getBlockLocations took 1ms
> 15/09/14 13:00:14 DEBUG FileInputFormat: Total # of splits generated by
> getSplits: 2, TimeTaken: 104
> ...
> 15/09/14 13:00:24 DEBUG Client: IPC Client (1739425103) connection to
> mesos-1.example.com/10.1.200.165:8020 from ubuntu: closed
> 15/09/14 13:00:24 DEBUG Client: IPC Client (1739425103) connection to
> mesos-1.example.com/10.1.200.165:8020 from ubuntu: stopped, remaining
> connections 0
> 15/09/14 13:00:24 DEBUG
> AkkaRpcEnv$$anonfun$actorRef$lzycompute$1$1$$anon$1: [actor] received
> message
> AkkaMessage(ExecutorRemoved(20150826-133446-3217621258-5050-4064-S1),true)
> from Actor[akka://sparkDriver/temp/$g]
> 15/09/14 13:00:24 DEBUG
> 

Re: hdfs-ha on mesos - odd bug

2015-09-15 Thread Adrian Bridgett
Thanks Steve - we are already taking the safe route - putting NN and 
datanodes on the central mesos-masters which are on demand.  Later (much 
later!) we _may_ put some datanodes on spot instances (and using several 
spot instance types as the spikes seem to only affect one type - worst 
case we can rebuild the data as well).  OTOH this would mainly only be 
beneficial if spark/mesos understood the data locality which is probably 
some time off (we don't need this ability now).


Indeed, the error we are seeing is orthogonal to the setup - however my 
understanding of ha-hdfs is that it should be resolved via the 
hdfs-site.xml file and doesn't use DNS whatsoever (and indeed, it _does_ 
work - but only after we initialise the driver with a bad hdfs url.)  I 
think there's some (missing) HDFS initialisation therefore when running 
spark on mesos - my suspicion is on the spark side (or my spark config).


http://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-hdfs/HDFSHighAvailabilityWithNFS.html#Configuration_details

On 15/09/2015 10:24, Steve Loughran wrote:

On 15 Sep 2015, at 08:55, Adrian Bridgett  wrote:

Hi Sam, in short, no, it's a traditional install as we plan to use spot 
instances and didn't want price spikes to kill off HDFS.

We're actually doing a bit of a hybrid, using spot instances for the mesos 
slaves, ondemand for the mesos masters.  So for the time being, putting hdfs on 
the masters (we'll probably move to multiple slave instance types to avoid 
losing too many when spot price spikes, but for now this is acceptable).   
Masters running CDH5.

It's incredibly dangerous using hdfs NNs on spot vms; a significant enough 
spike will lose all of them in one go, and there goes your entire filesystem. 
Have a static VM, maybe even backed by EBS.

If you look at Hadoop architectures from Hortonworks, Cloudera and Amazon 
themselves, the usual stance is HDFS on static nodes, spot instances for 
compute only


Using hdfs://current-hdfs-master:8020 works fine, however using 
hdfs://nameservice1 fails in the rather odd way described (well, more that the 
workaround actually works!)  I think there's some underlying bug here that's 
being exposed.


this sounds an issue orthogonal to spot instances. Maybe related to how JVMs 
cache DNS entries forever?


--
*Adrian Bridgett* |  Sysadmin Engineer, OpenSignal 


_
Office: First Floor, Scriptor Court, 155-157 Farringdon Road, 
Clerkenwell, London, EC1R 3AD

Phone #: +44 777-377-8251
Skype: abridgett  |@adrianbridgett | 
LinkedIn link 

_


hdfs-ha on mesos - odd bug

2015-09-14 Thread Adrian Bridgett
I'm hitting an odd issue with running spark on mesos together with 
HA-HDFS, with an even odder workaround.


In particular I get an error that it can't find the HDFS nameservice 
unless I put in a _broken_ url (discovered that workaround by 
mistake!).  core-site.xml, hdfs-site.xml is distributed to the slave 
node - and that file is read since I deliberately break the file then I 
get an error as you'd expect.


NB: This is a bit different to 
http://mail-archives.us.apache.org/mod_mbox/spark-user/201402.mbox/%3c1392442185079-1549.p...@n3.nabble.com%3E



Spark 1.5.0:

t=sc.textFile("hdfs://nameservice1/tmp/issue")
t.count()
(fails)

t=sc.textFile("file://etc/passwd")
t.count()
(errors about bad url - should have an extra / of course)
t=sc.textFile("hdfs://nameservice1/tmp/issue")
t.count()
then it works!!!

I should say that using file:///etc/passwd or hdfs:///tmp/issue both 
fail as well.  Unless preceded by a broken url.I've tried setting 
spark.hadoop.cloneConf to true, no change.


Sample (broken) run:
15/09/14 13:00:14 DEBUG HadoopRDD: Creating new JobConf and caching it 
for later re-use
15/09/14 13:00:14 DEBUG : address: ip-10-1-200-165/10.1.200.165 
isLoopbackAddress: false, with host 10.1.200.165 ip-10-1-200-165
15/09/14 13:00:14 DEBUG BlockReaderLocal: 
dfs.client.use.legacy.blockreader.local = false
15/09/14 13:00:14 DEBUG BlockReaderLocal: dfs.client.read.shortcircuit = 
false
15/09/14 13:00:14 DEBUG BlockReaderLocal: 
dfs.client.domain.socket.data.traffic = false
15/09/14 13:00:14 DEBUG BlockReaderLocal: dfs.domain.socket.path = 
/var/run/hdfs-sockets/dn
15/09/14 13:00:14 DEBUG HAUtil: No HA service delegation token found for 
logical URI hdfs://nameservice1
15/09/14 13:00:14 DEBUG BlockReaderLocal: 
dfs.client.use.legacy.blockreader.local = false
15/09/14 13:00:14 DEBUG BlockReaderLocal: dfs.client.read.shortcircuit = 
false
15/09/14 13:00:14 DEBUG BlockReaderLocal: 
dfs.client.domain.socket.data.traffic = false
15/09/14 13:00:14 DEBUG BlockReaderLocal: dfs.domain.socket.path = 
/var/run/hdfs-sockets/dn

15/09/14 13:00:14 DEBUG RetryUtils: multipleLinearRandomRetry = null
15/09/14 13:00:14 DEBUG Server: rpcKind=RPC_PROTOCOL_BUFFER, 
rpcRequestWrapperClass=class 
org.apache.hadoop.ipc.ProtobufRpcEngine$RpcRequestWrapper, 
rpcInvoker=org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker@6245f50b
15/09/14 13:00:14 DEBUG Client: getting client out of cache: 
org.apache.hadoop.ipc.Client@267f0fd3
15/09/14 13:00:14 DEBUG NativeCodeLoader: Trying to load the 
custom-built native-hadoop library...

15/09/14 13:00:14 DEBUG NativeCodeLoader: Loaded the native-hadoop library
...
15/09/14 13:00:14 DEBUG Client: Connecting to 
mesos-1.example.com/10.1.200.165:8020
15/09/14 13:00:14 DEBUG Client: IPC Client (1739425103) connection to 
mesos-1.example.com/10.1.200.165:8020 from ubuntu: starting, having 
connections 1
15/09/14 13:00:14 DEBUG Client: IPC Client (1739425103) connection to 
mesos-1.example.com/10.1.200.165:8020 from ubuntu sending #0
15/09/14 13:00:14 DEBUG Client: IPC Client (1739425103) connection to 
mesos-1.example.com/10.1.200.165:8020 from ubuntu got value #0

15/09/14 13:00:14 DEBUG ProtobufRpcEngine: Call: getFileInfo took 36ms
15/09/14 13:00:14 DEBUG FileInputFormat: Time taken to get FileStatuses: 69
15/09/14 13:00:14 INFO FileInputFormat: Total input paths to process : 1
15/09/14 13:00:14 DEBUG Client: IPC Client (1739425103) connection to 
mesos-1.example.com/10.1.200.165:8020 from ubuntu sending #1
15/09/14 13:00:14 DEBUG Client: IPC Client (1739425103) connection to 
mesos-1.example.com/10.1.200.165:8020 from ubuntu got value #1

15/09/14 13:00:14 DEBUG ProtobufRpcEngine: Call: getBlockLocations took 1ms
15/09/14 13:00:14 DEBUG FileInputFormat: Total # of splits generated by 
getSplits: 2, TimeTaken: 104

...
15/09/14 13:00:24 DEBUG Client: IPC Client (1739425103) connection to 
mesos-1.example.com/10.1.200.165:8020 from ubuntu: closed
15/09/14 13:00:24 DEBUG Client: IPC Client (1739425103) connection to 
mesos-1.example.com/10.1.200.165:8020 from ubuntu: stopped, remaining 
connections 0
15/09/14 13:00:24 DEBUG 
AkkaRpcEnv$$anonfun$actorRef$lzycompute$1$1$$anon$1: [actor] received 
message 
AkkaMessage(ExecutorRemoved(20150826-133446-3217621258-5050-4064-S1),true) 
from Actor[akka://sparkDriver/temp/$g]
15/09/14 13:00:24 DEBUG 
AkkaRpcEnv$$anonfun$actorRef$lzycompute$1$1$$anon$1: Received RPC 
message: 
AkkaMessage(ExecutorRemoved(20150826-133446-3217621258-5050-4064-S1),true)
15/09/14 13:00:24 DEBUG 
AkkaRpcEnv$$anonfun$actorRef$lzycompute$1$1$$anon$1: [actor] handled 
message (0.513851 ms) 
AkkaMessage(ExecutorRemoved(20150826-133446-3217621258-5050-4064-S1),true) 
from Actor[akka://sparkDriver/temp/$g]
15/09/14 13:00:25 WARN TaskSetManager: Lost task 0.0 in stage 0.0 (TID 
0, 10.1.200.245): java.lang.IllegalArgumentException: 
java.net.UnknownHostException: nameservice1
at 
org.apache.hadoop.security.SecurityUtil.buildTokenService(SecurityUtil.java:377)

Re: hdfs-ha on mesos - odd bug

2015-09-14 Thread Sam Bessalah
I don't know about the broken url. But are you running HDFS as a mesos
framework? If so is it using mesos-dns?
Then you should resolve the namenode via hdfs:/// 

On Mon, Sep 14, 2015 at 3:55 PM, Adrian Bridgett 
wrote:

> I'm hitting an odd issue with running spark on mesos together with
> HA-HDFS, with an even odder workaround.
>
> In particular I get an error that it can't find the HDFS nameservice
> unless I put in a _broken_ url (discovered that workaround by mistake!).
> core-site.xml, hdfs-site.xml is distributed to the slave node - and that
> file is read since I deliberately break the file then I get an error as
> you'd expect.
>
> NB: This is a bit different to
> http://mail-archives.us.apache.org/mod_mbox/spark-user/201402.mbox/%3c1392442185079-1549.p...@n3.nabble.com%3E
>
>
> Spark 1.5.0:
>
> t=sc.textFile("hdfs://nameservice1/tmp/issue")
> t.count()
> (fails)
>
> t=sc.textFile("file://etc/passwd")
> t.count()
> (errors about bad url - should have an extra / of course)
> t=sc.textFile("hdfs://nameservice1/tmp/issue")
> t.count()
> then it works!!!
>
> I should say that using file:///etc/passwd or hdfs:///tmp/issue both fail
> as well.  Unless preceded by a broken url.I've tried setting
> spark.hadoop.cloneConf to true, no change.
>
> Sample (broken) run:
> 15/09/14 13:00:14 DEBUG HadoopRDD: Creating new JobConf and caching it for
> later re-use
> 15/09/14 13:00:14 DEBUG : address: ip-10-1-200-165/10.1.200.165
> isLoopbackAddress: false, with host 10.1.200.165 ip-10-1-200-165
> 15/09/14 13:00:14 DEBUG BlockReaderLocal:
> dfs.client.use.legacy.blockreader.local = false
> 15/09/14 13:00:14 DEBUG BlockReaderLocal: dfs.client.read.shortcircuit =
> false
> 15/09/14 13:00:14 DEBUG BlockReaderLocal:
> dfs.client.domain.socket.data.traffic = false
> 15/09/14 13:00:14 DEBUG BlockReaderLocal: dfs.domain.socket.path =
> /var/run/hdfs-sockets/dn
> 15/09/14 13:00:14 DEBUG HAUtil: No HA service delegation token found for
> logical URI hdfs://nameservice1
> 15/09/14 13:00:14 DEBUG BlockReaderLocal:
> dfs.client.use.legacy.blockreader.local = false
> 15/09/14 13:00:14 DEBUG BlockReaderLocal: dfs.client.read.shortcircuit =
> false
> 15/09/14 13:00:14 DEBUG BlockReaderLocal:
> dfs.client.domain.socket.data.traffic = false
> 15/09/14 13:00:14 DEBUG BlockReaderLocal: dfs.domain.socket.path =
> /var/run/hdfs-sockets/dn
> 15/09/14 13:00:14 DEBUG RetryUtils: multipleLinearRandomRetry = null
> 15/09/14 13:00:14 DEBUG Server: rpcKind=RPC_PROTOCOL_BUFFER,
> rpcRequestWrapperClass=class
> org.apache.hadoop.ipc.ProtobufRpcEngine$RpcRequestWrapper,
> rpcInvoker=org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker@6245f50b
> 15/09/14 13:00:14 DEBUG Client: getting client out of cache:
> org.apache.hadoop.ipc.Client@267f0fd3
> 15/09/14 13:00:14 DEBUG NativeCodeLoader: Trying to load the custom-built
> native-hadoop library...
> 15/09/14 13:00:14 DEBUG NativeCodeLoader: Loaded the native-hadoop library
> ...
> 15/09/14 13:00:14 DEBUG Client: Connecting to
> mesos-1.example.com/10.1.200.165:8020
> 15/09/14 13:00:14 DEBUG Client: IPC Client (1739425103) connection to
> mesos-1.example.com/10.1.200.165:8020 from ubuntu: starting, having
> connections 1
> 15/09/14 13:00:14 DEBUG Client: IPC Client (1739425103) connection to
> mesos-1.example.com/10.1.200.165:8020 from ubuntu sending #0
> 15/09/14 13:00:14 DEBUG Client: IPC Client (1739425103) connection to
> mesos-1.example.com/10.1.200.165:8020 from ubuntu got value #0
> 15/09/14 13:00:14 DEBUG ProtobufRpcEngine: Call: getFileInfo took 36ms
> 15/09/14 13:00:14 DEBUG FileInputFormat: Time taken to get FileStatuses: 69
> 15/09/14 13:00:14 INFO FileInputFormat: Total input paths to process : 1
> 15/09/14 13:00:14 DEBUG Client: IPC Client (1739425103) connection to
> mesos-1.example.com/10.1.200.165:8020 from ubuntu sending #1
> 15/09/14 13:00:14 DEBUG Client: IPC Client (1739425103) connection to
> mesos-1.example.com/10.1.200.165:8020 from ubuntu got value #1
> 15/09/14 13:00:14 DEBUG ProtobufRpcEngine: Call: getBlockLocations took 1ms
> 15/09/14 13:00:14 DEBUG FileInputFormat: Total # of splits generated by
> getSplits: 2, TimeTaken: 104
> ...
> 15/09/14 13:00:24 DEBUG Client: IPC Client (1739425103) connection to
> mesos-1.example.com/10.1.200.165:8020 from ubuntu: closed
> 15/09/14 13:00:24 DEBUG Client: IPC Client (1739425103) connection to
> mesos-1.example.com/10.1.200.165:8020 from ubuntu: stopped, remaining
> connections 0
> 15/09/14 13:00:24 DEBUG
> AkkaRpcEnv$$anonfun$actorRef$lzycompute$1$1$$anon$1: [actor] received
> message
> AkkaMessage(ExecutorRemoved(20150826-133446-3217621258-5050-4064-S1),true)
> from Actor[akka://sparkDriver/temp/$g]
> 15/09/14 13:00:24 DEBUG
> AkkaRpcEnv$$anonfun$actorRef$lzycompute$1$1$$anon$1: Received RPC message:
> AkkaMessage(ExecutorRemoved(20150826-133446-3217621258-5050-4064-S1),true)
> 15/09/14 13:00:24 DEBUG
> AkkaRpcEnv$$anonfun$actorRef$lzycompute$1$1$$anon$1: [actor] handled
> message