hadoop-common-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Konstantin Boudnik <...@apache.org>
Subject Re: HDFS Backup nodes
Date Wed, 14 Dec 2011 07:28:39 GMT
On Tue, Dec 13, 2011 at 11:00PM, M. C. Srivas wrote:
> Suresh,
> 
> As of today, there is no option except to use NFS.  And as you yourself
> mention, the first HA prototype when it comes out will require NFS.

Well, in the interest of full disclosure NFS is just one of the options and
not the only one. Any auxiliary storage will do greatly. Distributed in-memory
redundant storage for sub-seconds fail-over? Sure, Gigaspaces do this for
years using very mature JINI.

NFS is just happen to be readily available in any data center and doesn't
require much of the extra investment on top of what exists. NFS comes with its
own set of problems of course. First and foremost is No-File-Security which
requires use of something like Kerberos for third-party user management. And
when paired with something like LinuxTaskController it can produce some very
interesting effects.

Cos

> (a) I wasn't aware that Bookkeeper had progressed that far. I wonder
> whether it would be able to keep up with the data rates that is required in
> order to hold the NN log without falling behind.
> 
> (b) I do know Karthik Ranga at FB just started a design to put the NN data
> in HDFS itself, but that is in very preliminary design stages with no real
> code there.
> 
> The problem is that the HA code written with NFS in mind is very different
> from the HA code written with HDFS in mind, which are both quite different
> from the code that is written with Bookkeeper in mind. Essentially the
> three options will form three different implementations, since the failure
> modes of each of the back-ends are different. Am I totally off base?
> 
> thanks,
> Srivas.
> 
> 
> 
> 
> On Tue, Dec 13, 2011 at 11:00 AM, Suresh Srinivas <suresh@hortonworks.com>wrote:
> 
> > Srivas,
> >
> > As you may know already, NFS is just being used in the first prototype for
> > HA.
> >
> > Two options for editlog store are:
> > 1. Using BookKeeper. Work has already completed on trunk towards this. This
> > will replace need for NFS to  store the editlogs and is highly available.
> > This solution will also be used for HA.
> > 2. We have a short term goal also to enable editlogs going to HDFS itself.
> > The work is in progress.
> >
> > Regards,
> > Suresh
> >
> >
> > >
> > > ---------- Forwarded message ----------
> > > From: M. C. Srivas <mcsrivas@gmail.com>
> > > Date: Sun, Dec 11, 2011 at 10:47 PM
> > > Subject: Re: HDFS Backup nodes
> > > To: common-user@hadoop.apache.org
> > >
> > >
> > > You are out of luck if you don't want to use NFS, and yet want redundancy
> > > for the NN.  Even the new "NN HA" work being done by the community will
> > > require NFS ... and the NFS itself needs to be HA.
> > >
> > > But if you use a Netapp, then the likelihood of the Netapp crashing is
> > > lower than the likelihood of a garbage-collection-of-death happening in
> > the
> > > NN.
> > >
> > > [ disclaimer:  I don't work for Netapp, I work for MapR ]
> > >
> > >
> > > On Wed, Dec 7, 2011 at 4:30 PM, randy <randysch@comcast.net> wrote:
> > >
> > > > Thanks Joey. We've had enough problems with nfs (mainly under very high
> > > > load) that we thought it might be riskier to use it for the NN.
> > > >
> > > > randy
> > > >
> > > >
> > > > On 12/07/2011 06:46 PM, Joey Echeverria wrote:
> > > >
> > > >> Hey Rand,
> > > >>
> > > >> It will mark that storage directory as failed and ignore it from then
> > > >> on. In order to do this correctly, you need a couple of options
> > > >> enabled on the NFS mount to make sure that it doesn't retry
> > > >> infinitely. I usually run with the tcp,soft,intr,timeo=10,**retrans=10
> > > >> options set.
> > > >>
> > > >> -Joey
> > > >>
> > > >> On Wed, Dec 7, 2011 at 12:37 PM,<randysch@comcast.net>  wrote:
> > > >>
> > > >>> What happens then if the nfs server fails or isn't reachable?
Does
> > hdfs
> > > >>> lock up? Does it gracefully ignore the nfs copy?
> > > >>>
> > > >>> Thanks,
> > > >>> randy
> > > >>>
> > > >>> ----- Original Message -----
> > > >>> From: "Joey Echeverria"<joey@cloudera.com>
> > > >>> To: common-user@hadoop.apache.org
> > > >>> Sent: Wednesday, December 7, 2011 6:07:58 AM
> > > >>> Subject: Re: HDFS Backup nodes
> > > >>>
> > > >>> You should also configure the Namenode to use an NFS mount for
one of
> > > >>> it's storage directories. That will give the most up-to-date back
of
> > > >>> the metadata in case of total node failure.
> > > >>>
> > > >>> -Joey
> > > >>>
> > > >>> On Wed, Dec 7, 2011 at 3:17 AM, praveenesh kumar<
> > praveenesh@gmail.com>
> > > >>>  wrote:
> > > >>>
> > > >>>> This means still we are relying on Secondary NameNode idealogy
for
> > > >>>> Namenode's backup.
> > > >>>> Can OS-mirroring of Namenode is a good alternative keep it
alive all
> > > the
> > > >>>> time ?
> > > >>>>
> > > >>>> Thanks,
> > > >>>> Praveenesh
> > > >>>>
> > > >>>> On Wed, Dec 7, 2011 at 1:35 PM, Uma Maheswara Rao G<
> > > >>>> maheswara@huawei.com>wrote:
> > > >>>>
> > > >>>>  AFAIK backup node introduced in 0.21 version onwards.
> > > >>>>> ______________________________**__________
> > > >>>>> From: praveenesh kumar [praveenesh@gmail.com]
> > > >>>>> Sent: Wednesday, December 07, 2011 12:40 PM
> > > >>>>> To: common-user@hadoop.apache.org
> > > >>>>> Subject: HDFS Backup nodes
> > > >>>>>
> > > >>>>> Does hadoop 0.20.205 supports configuring HDFS backup
nodes ?
> > > >>>>>
> > > >>>>> Thanks,
> > > >>>>> Praveenesh
> > > >>>>>
> > > >>>>>
> > > >>>
> > > >>>
> > > >>> --
> > > >>> Joseph Echeverria
> > > >>> Cloudera, Inc.
> > > >>> 443.305.9434
> > > >>>
> > > >>
> > > >>
> > > >>
> > > >>
> > > >
> > >
> > >
> >

Mime
View raw message