1. The Tool opens cursor to the database (RDBMS) and reads 'rowid's . We keep 
ROWIDs in RDBMS
2. Then using first N (configurable) rowids it fetches these rows from HBase 
using HBase bulk get API.

HBase restart :

stop-hbase
start-hbase

on Master node

I do not kill RS - its dangerous :)


Best regards,
Vladimir Rodionov
Principal Platform Engineer
Carrier IQ, www.carrieriq.com
e-mail: [email protected]

________________________________________
From: [email protected] [[email protected]] On Behalf Of Stack 
[[email protected]]
Sent: Wednesday, November 02, 2011 2:56 PM
To: [email protected]
Subject: Re: HBase 0.90.4 missing data in production

On Wed, Nov 2, 2011 at 1:24 PM, Vladimir Rodionov
<[email protected]> wrote:
> We can confirm that by running  our own internal tool.

Whats this tool doing Vladimir?  Is it running against HBase API?

> It seems, that we loose only during first restart
>

How are you doing the restart?  You killing regionservers?  On
restart, are we splitting wal logs or is it a 'clean' restart where no
wals are split?

Can you figure what the missing data is?  Is it all from same one or
two regions?


> 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.
>

I'd think that if your 'internal tool' found the stuff before the
restart, then this is probably ok (a year could have elapsed over the
restart I suppose but that be odd...)


> 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
>


Perhaps a newer CDH would have fix for this?


> This is probably not all ERRORS and FATALs I am continuing investigation and 
> will post my other findings later.
>

Let us know.

St.Ack

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.

Reply via email to