No, I restart only HBase cluster It seems that the FATAL error is the only one I can find in RS log files. No ERRORs
Best regards, Vladimir Rodionov Principal Platform Engineer Carrier IQ, www.carrieriq.com e-mail: [email protected] ________________________________________ From: Todd Lipcon [[email protected]] Sent: Wednesday, November 02, 2011 1:27 PM To: [email protected] Subject: Re: HBase 0.90.4 missing data in production Are you restarting your HDFS cluster underneath your running HBase cluster? Since you're running a rather old CDH3 (+320 is CDH3b2 if I remember correctly, something like 18mo old) you have a couple missing bug fixes that would cause data loss in the scenario that you restart HDFS under HBase. On Wed, Nov 2, 2011 at 1:24 PM, Vladimir Rodionov <[email protected]> wrote: > Even with WAL enabled > HBase version: 0.90.4 > Hadoop: 0.20.2+320 (CDH3) > > > When M/R job finishes loading data into HBase table and right after that > everything is OK, no missing data. We can confirm that by running our own > internal tool. > But when we re-start the HBase cluster and run the checker again we regularly > find missing rows in a table. The total number of rows decreases in a table. > This "missing rows" amount is small = ~ 0.1-0.3% of a total. After first > restart all subsequent restart do not affect on a total number of rows. It > seems, > that we loose only during first restart > > Table's TTL = 1 year. There is a slim chance that we load data with > timestamps more than one year behind, but it does not explain the difference > between total number of rows before and after cluster's restart. > > All RS are time synched. > > In Master log I do not see any WARN or ERRORs during cluster re-start. In RS > logs I see a lot of: > > 2011-11-02 00:16:07,620 INFO org.apache.hadoop.hbase.regionserver.wal.HLog: > Roll > /hbase/.logs/us01-ciqps1-grid01.carrieriq.com,60020,1320187507171/us01-ciqps1-grid01.carrieriq.com%3A60020.1320192949451, > entries=76, filesize=68053806. New hlog > /hbase/.logs/us01-ciqps1-grid01.carrieriq.com,60020,1320187507171/us01-ciqps1-grid01.carrieriq.com%3A60020.1320192967380 > 2011-11-02 00:16:07,621 INFO org.apache.hadoop.io.compress.CodecPool: Got > brand-new compressor > 2011-11-02 00:16:07,621 FATAL org.apache.hadoop.hbase.regionserver.wal.HLog: > Could not append. Requesting close of hlog > java.io.IOException: Reflection > at > org.apache.hadoop.hbase.regionserver.wal.SequenceFileLogWriter.sync(SequenceFileLogWriter.java:147) > at org.apache.hadoop.hbase.regionserver.wal.HLog.sync(HLog.java:1002) > at org.apache.hadoop.hbase.regionserver.wal.HLog.append(HLog.java:955) > at > org.apache.hadoop.hbase.regionserver.HRegion.doMiniBatchPut(HRegion.java:1483) > at org.apache.hadoop.hbase.regionserver.HRegion.put(HRegion.java:1392) > at > org.apache.hadoop.hbase.regionserver.HRegionServer.multi(HRegionServer.java:2591) > at sun.reflect.GeneratedMethodAccessor13.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.ipc.HBaseRPC$Server.call(HBaseRPC.java:570) > at > org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:1039) > Caused by: java.lang.reflect.InvocationTargetException > at sun.reflect.GeneratedMethodAccessor2.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.sync(SequenceFileLogWriter.java:145) > ... 10 more > Caused by: java.lang.NullPointerException > at > org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.closeThreads(DFSClient.java:3306) > at > org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.sync(DFSClient.java:3216) > at > org.apache.hadoop.fs.FSDataOutputStream.sync(FSDataOutputStream.java:97) > at > org.apache.hadoop.io.SequenceFile$Writer.syncFs(SequenceFile.java:944) > ... 14 more > > This is probably not all ERRORS and FATALs I am continuing investigation and > will post my other findings later. > > Can someone tell me what should I check else? > > Best regards, > Vladimir Rodionov > Principal Platform Engineer > Carrier IQ, www.carrieriq.com > e-mail: [email protected] > > ________________________________________ > From: Todd Lipcon [[email protected]] > Sent: Wednesday, November 02, 2011 12:53 PM > To: [email protected] > Subject: Re: HBASE-4120 table level priority > > It's up to each committer to decide how they want to allocate their > time, of course, but I'd like to encourage folks to hold off on > spending a lot of time on new 0.94 feature merges until we've gotten > 0.92.0 stabilized and out the door. Big features like QoS need a fair > amount of review, both on the general approach and on the specific > implementation, before they can get merged. It's hard to find time for > a thorough review when we're in the finishing stages of stabilizing a > release. > > -Todd > > On Wed, Nov 2, 2011 at 12:23 PM, Ted Yu <[email protected]> wrote: >> Hi, >> HBASE-4120 is a new feature for 0.94 that allows users to set different >> priorities for their tables. >> Priorities can also be assigned to methods (get's vs. put's). >> >> The new tests in the patch all pass. >> >> Please review the latest patch and provide your comments. >> >> Thanks >> > > > > -- > Todd Lipcon > Software Engineer, Cloudera > > Confidentiality Notice: The information contained in this message, including > any attachments hereto, may be confidential and is intended to be read only > by the individual or entity to whom this message is addressed. If the reader > of this message is not the intended recipient or an agent or designee of the > intended recipient, please note that any review, use, disclosure or > distribution of this message or its attachments, in any form, is strictly > prohibited. If you have received this message in error, please immediately > notify the sender and/or [email protected] and delete or destroy > any copy of this message and its attachments. > -- Todd Lipcon Software Engineer, Cloudera Confidentiality Notice: The information contained in this message, including any attachments hereto, may be confidential and is intended to be read only by the individual or entity to whom this message is addressed. If the reader of this message is not the intended recipient or an agent or designee of the intended recipient, please note that any review, use, disclosure or distribution of this message or its attachments, in any form, is strictly prohibited. If you have received this message in error, please immediately notify the sender and/or [email protected] and delete or destroy any copy of this message and its attachments.
