hbase-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Bharath Vissapragada <bhara...@cloudera.com>
Subject Re: Hbase - hadoop handling DNS blips
Date Wed, 27 Aug 2014 07:26:56 GMT
Hi Arun,

My 2cents.

I've seen this sometime in the past and after doing some research, the
issue seems to be related to
https://issues.apache.org/jira/browse/HADOOP-6356 . HLog (SequenceFile)
internally uses FileContext( unlike other HBase components which use
FileSystem), which doesn't cache connections. So every time you create a
log file writer instance, it eventually hits DNS and crashes.

- Bharath


On Wed, Aug 27, 2014 at 12:40 PM, Arun Mishra <arunmishra.h@gmail.com>
wrote:

> Hello,
>
> This is the first time I am sending a query on hbase mailing list.
> Hopefully this is the correct group to ask hbase/hadoop related questions.
>
> I am running hbase 0.92, hadoop 2.0 (cdh 4.1.3). Recently, there were some
> instability in my dns service and host lookups request failed occasionally.
> During such failures, some random region server will shut itself down when
> it encounters a fatal exception during log roll operation. DNS issue was
> eventually resolved and region server fatals stopped.
>
> While I was trying to understand the hbase/hadoop behavior during network
> events/blips, I found there is a default retry policy used -
> TRY_ONCE_THEN_FAIL. Please correct me if thats not the case.
>
> But then I was thinking that there could be more of these blips during
> network or some other infrastructure maintenance operations. These
> maintenance operations should not result in region server going down. If
> the client simply attempts one more time, host lookup request should
> succeed.
>
> If someone has any similar experience, can they please share? Are there
> options one can try out against such failures?
>
> May be I am not thinking in the right direction, but this behavior makes
> me feel that hbase (using hdfs) is sensitive to DNS service availability.
> DNS unavailability for even few seconds can bring down the entire cluster
> (rare chance if all attempt to roll hlogs at the same time).
>
> Here is the stack trace:
> 11:14:48.706 AM
> 2014-08-17 11:14:48,706 FATAL
> org.apache.hadoop.hbase.regionserver.HRegionServer: ABORTING region server
> hadoop0104111601,60020,1408273008941: IOE in log roller
> java.io.IOException: cannot get log writer
>         at
> org.apache.hadoop.hbase.regionserver.wal.HLog.createWriter(HLog.java:716)
>         at
> org.apache.hadoop.hbase.regionserver.wal.HLog.createWriterInstance(HLog.java:663)
>         at
> org.apache.hadoop.hbase.regionserver.wal.HLog.rollWriter(HLog.java:595)
>         at
> org.apache.hadoop.hbase.regionserver.LogRoller.run(LogRoller.java:94)
>         at java.lang.Thread.run(Thread.java:662)
> Caused by: java.io.IOException: java.lang.RuntimeException:
> java.lang.reflect.InvocationTargetException
>         at
> org.apache.hadoop.hbase.regionserver.wal.SequenceFileLogWriter.init(SequenceFileLogWriter.java:106)
>         at
> org.apache.hadoop.hbase.regionserver.wal.HLog.createWriter(HLog.java:713)
>         ... 4 more
> Caused by: java.lang.RuntimeException:
> java.lang.reflect.InvocationTargetException
>         at
> org.apache.hadoop.fs.AbstractFileSystem.newInstance(AbstractFileSystem.java:122)
>         at
> org.apache.hadoop.fs.AbstractFileSystem.createFileSystem(AbstractFileSystem.java:148)
>         at
> org.apache.hadoop.fs.AbstractFileSystem.get(AbstractFileSystem.java:233)
>         at org.apache.hadoop.fs.FileContext$2.run(FileContext.java:321)
>         at org.apache.hadoop.fs.FileContext$2.run(FileContext.java:319)
>         at java.security.AccessController.doPrivileged(Native Method)
>         at javax.security.auth.Subject.doAs(Subject.java:396)
>         at
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1332)
>         at
> org.apache.hadoop.fs.FileContext.getAbstractFileSystem(FileContext.java:319)
>         at
> org.apache.hadoop.fs.FileContext.getFileContext(FileContext.java:432)
>         at
> org.apache.hadoop.io.SequenceFile.createWriter(SequenceFile.java:469)
>         at sun.reflect.GeneratedMethodAccessor38.invoke(Unknown Source)
>         at
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>         at java.lang.reflect.Method.invoke(Method.java:597)
>         at
> org.apache.hadoop.hbase.regionserver.wal.SequenceFileLogWriter.init(SequenceFileLogWriter.java:87)
>         ... 5 more
> Caused by: java.lang.reflect.InvocationTargetException
>         at sun.reflect.GeneratedConstructorAccessor20.newInstance(Unknown
> Source)
>         at
> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:27)
>         at java.lang.reflect.Constructor.newInstance(Constructor.java:513)
>         at
> org.apache.hadoop.fs.AbstractFileSystem.newInstance(AbstractFileSystem.java:120)
>         ... 19 more
> Caused by: java.lang.IllegalArgumentException:
> java.net.UnknownHostException: hadoop0104111601
>         at
> org.apache.hadoop.security.SecurityUtil.buildTokenService(SecurityUtil.java:414)
>         at
> org.apache.hadoop.hdfs.NameNodeProxies.createNonHAProxy(NameNodeProxies.java:164)
>         at
> org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider.getProxy(ConfiguredFailoverProxyProvider.java:125)
>         at
> org.apache.hadoop.io.retry.RetryInvocationHandler.<init>(RetryInvocationHandler.java:60)
>         at
> org.apache.hadoop.io.retry.RetryInvocationHandler.<init>(RetryInvocationHandler.java:51)
>         at org.apache.hadoop.io.retry.RetryProxy.create(RetryProxy.java:58)
>         at
> org.apache.hadoop.hdfs.NameNodeProxies.createProxy(NameNodeProxies.java:137)
>         at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:389)
>         at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:356)
>         at org.apache.hadoop.fs.Hdfs.<init>(Hdfs.java:84)
>         ... 23 more
> Caused by: java.net.UnknownHostException: hadoop0104111601
>         ... 33 more
>
> thanks,
> Arun




-- 
Bharath Vissapragada
<http://www.cloudera.com>

Mime
  • Unnamed multipart/alternative (inline, None, 0 bytes)
View raw message