Btw - I figured out the problem.

The jobconf from the remote client had the socks proxy configuration - the jvm 
spawned by TTs picked this up and tried to connect using the proxy which of 
course didn't work.

This was easy to solve - just had to make the remote initialization script mark 
hadoop.rpc.socket.factory.class.default as final variable in the 
hadoop-site.xml on server side.

I am assuming that this would be a good thing to do in general (can't believe 
why server side traffic would be routed through a proxy!). 

Filed https://issues.apache.org/jira/browse/HADOOP-5839 to follow up on the 
issues uncovered here.

-----Original Message-----
From: Tom White [mailto:t...@cloudera.com] 
Sent: Thursday, May 14, 2009 7:07 AM
To: core-user@hadoop.apache.org
Subject: Re: public IP for datanode on EC2

Yes, you're absolutely right.

Tom

On Thu, May 14, 2009 at 2:19 PM, Joydeep Sen Sarma <jssa...@facebook.com> wrote:
> The ec2 documentation point to the use of public 'ip' addresses - whereas 
> using public 'hostnames' seems safe since it resolves to internal addresses 
> from within the cluster (and resolve to public ip addresses from outside).
>
> The only data transfer that I would incur while submitting jobs from outside 
> is the cost of copying the jar files and any other files meant for the 
> distributed cache). That would be extremely small.
>
>
> -----Original Message-----
> From: Tom White [mailto:t...@cloudera.com]
> Sent: Thursday, May 14, 2009 5:58 AM
> To: core-user@hadoop.apache.org
> Subject: Re: public IP for datanode on EC2
>
> Hi Joydeep,
>
> The problem you are hitting may be because port 50001 isn't open,
> whereas from within the cluster any node may talk to any other node
> (because the security groups are set up to do this).
>
> However I'm not sure this is a good approach. Configuring Hadoop to
> use public IP addresses everywhere should work, but you have to pay
> for all data transfer between nodes (see http://aws.amazon.com/ec2/,
> "Public and Elastic IP Data Transfer"). This is going to get expensive
> fast!
>
> So to get this to work well, we would have to make changes to Hadoop
> so it was aware of both public and private addresses, and use the
> appropriate one: clients would use the public address, while daemons
> would use the private address. I haven't looked at what it would take
> to do this or how invasive it would be.
>
> Cheers,
> Tom
>
> On Thu, May 14, 2009 at 1:37 PM, Joydeep Sen Sarma <jssa...@facebook.com> 
> wrote:
>> I changed the ec2 scripts to have fs.default.name assigned to the public 
>> hostname (instead of the private hostname).
>>
>> Now I can submit jobs remotely via the socks proxy (the problem below is 
>> resolved) - but the map tasks fail with an exception:
>>
>>
>> 2009-05-14 07:30:34,913 INFO org.apache.hadoop.ipc.Client: Retrying connect 
>> to server: ec2-75-101-199-45.compute-1.amazonaws.com/10.254.175.132:50001. 
>> Already tried 9 time(s).
>> 2009-05-14 07:30:34,914 WARN org.apache.hadoop.mapred.TaskTracker: Error 
>> running child
>> java.io.IOException: Call to 
>> ec2-75-101-199-45.compute-1.amazonaws.com/10.254.175.132:50001 failed on 
>> local exception: Connection refused
>>        at org.apache.hadoop.ipc.Client.call(Client.java:699)
>>        at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:216)
>>        at $Proxy1.getProtocolVersion(Unknown Source)
>>        at org.apache.hadoop.ipc.RPC.getProxy(RPC.java:319)
>>        at 
>> org.apache.hadoop.hdfs.DFSClient.createRPCNamenode(DFSClient.java:104)
>>        at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:177)
>>        at 
>> org.apache.hadoop.hdfs.DistributedFileSystem.initialize(DistributedFileSystem.java:74)
>>        at 
>> org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:1367)
>>        at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:56)
>>        at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:1379)
>>        at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:215)
>>        at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:120)
>>        at org.apache.hadoop.mapred.Child.main(Child.java:153)
>>
>>
>> strangely enough - job submissions from nodes within the ec2 cluster work 
>> just fine. I looked at the job.xml files of jobs submitted locally and 
>> remotely and don't see any relevant differences.
>>
>> Totally foxed now.
>>
>> Joydeep
>>
>> -----Original Message-----
>> From: Joydeep Sen Sarma [mailto:jssa...@facebook.com]
>> Sent: Wednesday, May 13, 2009 9:38 PM
>> To: core-user@hadoop.apache.org
>> Cc: Tom White
>> Subject: RE: public IP for datanode on EC2
>>
>> Thanks Philip. Very helpful (and great blog post)! This seems to make basic 
>> dfs command line operations work just fine.
>>
>> However - I am hitting a new error during job submission (running 
>> hadoop-0.19.0):
>>
>> 2009-05-14 00:15:34,430 ERROR exec.ExecDriver 
>> (SessionState.java:printError(279)) - Job Submission failed with exception 
>> 'java.net.UnknownHostException(unknown host: 
>> domU-12-31-39-00-51-94.compute-1.internal)'
>> java.net.UnknownHostException: unknown host: 
>> domU-12-31-39-00-51-94.compute-1.internal
>>        at org.apache.hadoop.ipc.Client$Connection.<init>(Client.java:195)
>>        at org.apache.hadoop.ipc.Client.getConnection(Client.java:791)
>>        at org.apache.hadoop.ipc.Client.call(Client.java:686)
>>        at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:216)
>>        at $Proxy0.getProtocolVersion(Unknown Source)
>>        at org.apache.hadoop.ipc.RPC.getProxy(RPC.java:348)
>>        at 
>> org.apache.hadoop.hdfs.DFSClient.createRPCNamenode(DFSClient.java:104)
>>        at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:176)
>>        at 
>> org.apache.hadoop.hdfs.DistributedFileSystem.initialize(DistributedFileSystem.java:75)
>>        at 
>> org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:1367)
>>        at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:56)
>>        at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:1379)
>>        at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:215)
>>        at org.apache.hadoop.fs.Path.getFileSystem(Path.java:175)
>>        at org.apache.hadoop.mapred.JobClient.getFs(JobClient.java:469)
>>        at 
>> org.apache.hadoop.mapred.JobClient.configureCommandLineOptions(JobClient.java:603)
>>        at org.apache.hadoop.mapred.JobClient.submitJob(JobClient.java:788)
>>
>>
>> looking at the stack trace and the code - it seems that this is happening 
>> because the jobclient asks for the mapred system directory from the 
>> jobtracker - which replies back with a path name that's qualified against 
>> the fs.default.name setting of the jobtracker. Unfortunately the standard 
>> EC2 scripts assign this to the internal hostname of the hadoop master.
>>
>> Is there any downside to using public hostnames instead of the private ones 
>> in the ec2 starter scripts?
>>
>> Thanks for the help,
>>
>> Joydeep
>>
>>
>> -----Original Message-----
>> From: Philip Zeyliger [mailto:phi...@cloudera.com]
>> Sent: Wednesday, May 13, 2009 2:40 PM
>> To: core-user@hadoop.apache.org
>> Subject: Re: public IP for datanode on EC2
>>
>> On Tue, May 12, 2009 at 9:11 PM, Joydeep Sen Sarma <jssa...@facebook.com> 
>> wrote:
>>> (raking up real old thread)
>>>
>>> After struggling with this issue for sometime now - it seems that accessing 
>>> hdfs on ec2 from outside ec2 is not possible.
>>>
>>> This is primarily because of 
>>> https://issues.apache.org/jira/browse/HADOOP-985. Even if datanode ports 
>>> are authorized in ec2 and we set the public hostname via slave.host.name - 
>>> the namenode uses the internal IP address of the datanodes for block 
>>> locations. DFS clients outside ec2 cannot reach these addresses and report 
>>> failure reading/writing data blocks.
>>>
>>> HDFS/EC2 gurus - would it be reasonable to ask for an option to not use IP 
>>> addresses (and use datanode host names as pre-985)?
>>>
>>> I really like the idea of being able to use an external node (my personal 
>>> workstation) to do job submission (which typically requires interacting 
>>> with HDFS in order to push files into the jobcache etc). This way I don't 
>>> need custom AMIs - I can use stock hadoop amis (all the custom software is 
>>> on the external node). Without the above option - this is not possible 
>>> currently.
>>
>> You could use ssh to set up a SOCKS proxy between your machine and
>> ec2, and setup org.apache.hadoop.net.SocksSocketFactory to be the
>> socket factory.
>> http://www.cloudera.com/blog/2008/12/03/securing-a-hadoop-cluster-through-a-gateway/
>> has more information.
>>
>> -- Philip
>>
>

Reply via email to