You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-user@hadoop.apache.org by Derek Young <dy...@kayak.com> on 2009/01/21 22:23:56 UTC
using distcp for http source files
I plan to use hadoop to do some log processing and I'm working on a
method to load the files (probably nightly) into hdfs. My plan is to
have a web server on each machine with logs that serves up the log
directories. Then I would give distcp a list of http URLs of the log
files and have it copy the files in.
Reading http://issues.apache.org/jira/browse/HADOOP-341 it sounds like
this should be supported, but the http URLs are not working for me. Are
http source URLs still supported?
I tried a simple test with an http source URL (using Hadoop 0.19):
hadoop distcp -f http://core:7274/logs/log.20090121 /user/dyoung/mylogs
This fails:
With failures, global counters are inaccurate; consider running with -i
Copy failed: java.io.IOException: No FileSystem for scheme: http
at
org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:1364)
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.tools.DistCp.fetchFileList(DistCp.java:578)
at org.apache.hadoop.tools.DistCp.access$300(DistCp.java:74)
at org.apache.hadoop.tools.DistCp$Arguments.valueOf(DistCp.java:775)
at org.apache.hadoop.tools.DistCp.run(DistCp.java:844)
at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:65)
at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:79)
at org.apache.hadoop.tools.DistCp.main(DistCp.java:871)
Re: using distcp for http source files
Posted by Derek Young <dy...@kayak.com>.
Tsz Wo (Nicholas), Sze <s2...@...> writes:
>
> Hi Derek,
>
> The "http" in "http://core:7274/logs/log.20090121" should be "hftp". hftp is
the scheme name of
> HftpFileSystem which uses http for accessing hdfs.
>
> Hope this helps.
>
> Nicholas Sze
I thought hftp is used to talk to servlets that act as a gateway to hdfs
right? In my case these will be servers that are serving up static log files,
running no servlets. I believe this is the scenario that HADOOP-341 describes:
"Enhance it [distcp] to handle http as the source protocol i.e. support copying
files from arbitrary http-based sources into the dfs."
In any case if I just use hftp instead of http I get this error:
bin/hadoop distcp -f hftp://core:7274/logs/log.20090121 /user/dyoung/mylogs
With failures, global counters are inaccurate; consider running with -i
Copy failed: java.io.IOException: Server returned HTTP response code: 400 for
URL: http://core:7274/data/logs/log.20090121?
ugi=dyoung,dyoung,adm,dialout,fax,cdrom,cdrom,\
floppy,floppy,tape,audio,audio,dip,dip,video,video,\
plugdev,plugdev,admin,users,scanner,fuse,fuse,lpadmin,\
admin,vboxusers
at sun.net.www.protocol.http.HttpURLConnection.getInputStream
(HttpURLConnection.java:1241)
at org.apache.hadoop.hdfs.HftpFileSystem.open(HftpFileSystem.java:124)
at org.apache.hadoop.fs.FileSystem.open(FileSystem.java:359)
at org.apache.hadoop.tools.DistCp.fetchFileList(DistCp.java:581)
at org.apache.hadoop.tools.DistCp.access$300(DistCp.java:74)
at org.apache.hadoop.tools.DistCp$Arguments.valueOf(DistCp.java:775)
at org.apache.hadoop.tools.DistCp.run(DistCp.java:844)
at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:65)
at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:79)
at org.apache.hadoop.tools.DistCp.main(DistCp.java:871)
>
> ----- Original Message ----
> > From: Derek Young <dy...@...>
> > To: core-user@...
> > Sent: Wednesday, January 21, 2009 1:23:56 PM
> > Subject: using distcp for http source files
> >
> > I plan to use hadoop to do some log processing and I'm working on a method
to
> > load the files (probably nightly) into hdfs. My plan is to have a web
server on
> > each machine with logs that serves up the log directories. Then I would
give
> > distcp a list of http URLs of the log files and have it copy the files in.
> >
> > Reading http://issues.apache.org/jira/browse/HADOOP-341 it sounds like this
> > should be supported, but the http URLs are not working for me. Are http
source
> > URLs still supported?
> >
> > I tried a simple test with an http source URL (using Hadoop 0.19):
> >
> > hadoop distcp -f http://core:7274/logs/log.20090121 /user/dyoung/mylogs
> >
> > This fails:
> >
> > With failures, global counters are inaccurate; consider running with -i
> > Copy failed: java.io.IOException: No FileSystem for scheme: http
> > at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:1364)
> > 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.tools.DistCp.fetchFileList(DistCp.java:578)
> > at org.apache.hadoop.tools.DistCp.access$300(DistCp.java:74)
> > at org.apache.hadoop.tools.DistCp$Arguments.valueOf(DistCp.java:775)
> > at org.apache.hadoop.tools.DistCp.run(DistCp.java:844)
> > at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:65)
> > at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:79)
> > at org.apache.hadoop.tools.DistCp.main(DistCp.java:871)
>
>
Re: using distcp for http source files
Posted by "Tsz Wo (Nicholas), Sze" <s2...@yahoo.com>.
Hi Derek,
The "http" in "http://core:7274/logs/log.20090121" should be "hftp". hftp is the scheme name of HftpFileSystem which uses http for accessing hdfs.
Hope this helps.
Nicholas Sze
----- Original Message ----
> From: Derek Young <dy...@kayak.com>
> To: core-user@hadoop.apache.org
> Sent: Wednesday, January 21, 2009 1:23:56 PM
> Subject: using distcp for http source files
>
> I plan to use hadoop to do some log processing and I'm working on a method to
> load the files (probably nightly) into hdfs. My plan is to have a web server on
> each machine with logs that serves up the log directories. Then I would give
> distcp a list of http URLs of the log files and have it copy the files in.
>
> Reading http://issues.apache.org/jira/browse/HADOOP-341 it sounds like this
> should be supported, but the http URLs are not working for me. Are http source
> URLs still supported?
>
> I tried a simple test with an http source URL (using Hadoop 0.19):
>
> hadoop distcp -f http://core:7274/logs/log.20090121 /user/dyoung/mylogs
>
> This fails:
>
> With failures, global counters are inaccurate; consider running with -i
> Copy failed: java.io.IOException: No FileSystem for scheme: http
> at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:1364)
> 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.tools.DistCp.fetchFileList(DistCp.java:578)
> at org.apache.hadoop.tools.DistCp.access$300(DistCp.java:74)
> at org.apache.hadoop.tools.DistCp$Arguments.valueOf(DistCp.java:775)
> at org.apache.hadoop.tools.DistCp.run(DistCp.java:844)
> at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:65)
> at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:79)
> at org.apache.hadoop.tools.DistCp.main(DistCp.java:871)
Re: using distcp for http source files
Posted by Doug Cutting <cu...@apache.org>.
Can you please attach your latest version of this to
https://issues.apache.org/jira/browse/HADOOP-496?
Thanks,
Doug
Boris Musykantski wrote:
> we have fixed some patches in JIRA for support of webdav server on
> top of HDFS, updated to work with newer version (0.18.0 IIRC) and
> added support for permissions. See code and description here:
>
> http://www.hadoop.iponweb.net/Home/hdfs-over-webdav
>
> Hope it is useful,
>
> Regards,
> Boris, IPonWeb
>
> On Thu, Jan 22, 2009 at 2:30 PM, Doug Cutting <cu...@apache.org> wrote:
>> Aaron Kimball wrote:
>>> Is anyone aware of an OSS web dav library that
>>> could be wrapped in a FileSystem implementation?
>> We'd need a Java WebDAV client to talk to foreign filesystems. But to
>> expose HDFS to foreign filesystems (i.e., to better support mounting HDFS)
>> we'd need a Java WebDAV server, like http://milton.ettrema.com/.
>>
>> Doug
>>
Re: using distcp for http source files
Posted by Boris Musykantski <bo...@iponweb.net>.
we have fixed some patches in JIRA for support of webdav server on
top of HDFS, updated to work with newer version (0.18.0 IIRC) and
added support for permissions. See code and description here:
http://www.hadoop.iponweb.net/Home/hdfs-over-webdav
Hope it is useful,
Regards,
Boris, IPonWeb
On Thu, Jan 22, 2009 at 2:30 PM, Doug Cutting <cu...@apache.org> wrote:
> Aaron Kimball wrote:
>>
>> Is anyone aware of an OSS web dav library that
>> could be wrapped in a FileSystem implementation?
>
> We'd need a Java WebDAV client to talk to foreign filesystems. But to
> expose HDFS to foreign filesystems (i.e., to better support mounting HDFS)
> we'd need a Java WebDAV server, like http://milton.ettrema.com/.
>
> Doug
>
Re: using distcp for http source files
Posted by Doug Cutting <cu...@apache.org>.
Aaron Kimball wrote:
> Is anyone aware of an OSS web dav library that
> could be wrapped in a FileSystem implementation?
We'd need a Java WebDAV client to talk to foreign filesystems. But to
expose HDFS to foreign filesystems (i.e., to better support mounting
HDFS) we'd need a Java WebDAV server, like http://milton.ettrema.com/.
Doug
Re: Distributed cache testing in local mode
Posted by Tom White <to...@cloudera.com>.
It would be nice to make this more uniform. There's an outstanding
Jira on this if anyone is interested in looking at it:
https://issues.apache.org/jira/browse/HADOOP-2914
Tom
On Fri, Jan 23, 2009 at 12:14 AM, Aaron Kimball <aa...@cloudera.com> wrote:
> Hi Bhupesh,
>
> I've noticed the same problem -- LocalJobRunner makes the DistributedCache
> effectively not work; so my code often winds up with two codepaths to
> retrieve the local data :\
>
> You could try running in pseudo-distributed mode to test, though then you
> lose the ability to run a single-stepping debugger on the whole end-to-end
> process.
>
> - Aaron
>
> On Thu, Jan 22, 2009 at 11:29 AM, Bhupesh Bansal <bb...@linkedin.com>wrote:
>
>> Hey folks,
>>
>> I am trying to use Distributed cache in hadoop jobs to pass around
>> configuration files , external-jars (job sepecific) and some archive data.
>>
>> I want to test Job end-to-end in local mode, but I think the distributed
>> caches are localized in TaskTracker code which is not called in local mode
>> Through LocalJobRunner.
>>
>> I can do some fairly simple workarounds for this but was just wondering if
>> folks have more ideas about it.
>>
>> Thanks
>> Bhupesh
>>
>>
>
Re: Distributed cache testing in local mode
Posted by Aaron Kimball <aa...@cloudera.com>.
Hi Bhupesh,
I've noticed the same problem -- LocalJobRunner makes the DistributedCache
effectively not work; so my code often winds up with two codepaths to
retrieve the local data :\
You could try running in pseudo-distributed mode to test, though then you
lose the ability to run a single-stepping debugger on the whole end-to-end
process.
- Aaron
On Thu, Jan 22, 2009 at 11:29 AM, Bhupesh Bansal <bb...@linkedin.com>wrote:
> Hey folks,
>
> I am trying to use Distributed cache in hadoop jobs to pass around
> configuration files , external-jars (job sepecific) and some archive data.
>
> I want to test Job end-to-end in local mode, but I think the distributed
> caches are localized in TaskTracker code which is not called in local mode
> Through LocalJobRunner.
>
> I can do some fairly simple workarounds for this but was just wondering if
> folks have more ideas about it.
>
> Thanks
> Bhupesh
>
>
Distributed cache testing in local mode
Posted by Bhupesh Bansal <bb...@linkedin.com>.
Hey folks,
I am trying to use Distributed cache in hadoop jobs to pass around
configuration files , external-jars (job sepecific) and some archive data.
I want to test Job end-to-end in local mode, but I think the distributed
caches are localized in TaskTracker code which is not called in local mode
Through LocalJobRunner.
I can do some fairly simple workarounds for this but was just wondering if
folks have more ideas about it.
Thanks
Bhupesh
Re: using distcp for http source files
Posted by Doug Cutting <cu...@apache.org>.
Aaron Kimball wrote:
> Doesn't the WebDAV protocol use http for file transfer, and support reads /
> writes / listings / etc?
Yes. Getting a WebDAV-based FileSystem in Hadoop has long been a goal.
It could replace libhdfs, since there are already a WebDav-based FUSE
filesystem for Linux (wdfs, davfs2). WebDAV is also mountable from
Windows, etc.
> Is anyone aware of an OSS web dav library that
> could be wrapped in a FileSystem implementation?
Yes, Apache Slide does but it's dead. Apache Jackrabbit also does and
it is alive (http://jackrabbit.apache.org/).
Doug
Re: using distcp for http source files
Posted by Aaron Kimball <aa...@cloudera.com>.
Doesn't the WebDAV protocol use http for file transfer, and support reads /
writes / listings / etc? Is anyone aware of an OSS web dav library that
could be wrapped in a FileSystem implementation?
- Aaron
On Wed, Jan 21, 2009 at 3:30 PM, Doug Cutting <cu...@apache.org> wrote:
> Derek Young wrote:
>
>> Reading http://issues.apache.org/jira/browse/HADOOP-341 it sounds like
>> this should be supported, but the http URLs are not working for me. Are
>> http source URLs still supported?
>>
>
> No. They used to be supported, but when distcp was converted to accept any
> Path this stopped working, since there is no FileSystem implementation
> mapped to http: paths. Implementing an HttpFileSystem that supports
> read-only access to files and no directory listings is fairly trivial, but
> without directory listings, distcp would not work well.
>
> https://issues.apache.org/jira/browse/HADOOP-1563 includes a now
> long-stale patch that implements an HTTP filesystem, where directory
> listings are implemented, assuming that:
> - directories are represented by slash-terminated urls;
> - GET of a directory contains the URLs of its children
> This works for the directory listings returned by many HTTP servers.
>
> Perhaps someone can update this patch, and, if folks find it useful, we can
> include it.
>
> Doug
>
Re: using distcp for http source files
Posted by Doug Cutting <cu...@apache.org>.
Derek Young wrote:
> Reading http://issues.apache.org/jira/browse/HADOOP-341 it sounds like
> this should be supported, but the http URLs are not working for me. Are
> http source URLs still supported?
No. They used to be supported, but when distcp was converted to accept
any Path this stopped working, since there is no FileSystem
implementation mapped to http: paths. Implementing an HttpFileSystem
that supports read-only access to files and no directory listings is
fairly trivial, but without directory listings, distcp would not work well.
https://issues.apache.org/jira/browse/HADOOP-1563 includes a now
long-stale patch that implements an HTTP filesystem, where directory
listings are implemented, assuming that:
- directories are represented by slash-terminated urls;
- GET of a directory contains the URLs of its children
This works for the directory listings returned by many HTTP servers.
Perhaps someone can update this patch, and, if folks find it useful, we
can include it.
Doug