hbase-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From PG <pengyunm...@gmail.com>
Subject Re: How to config hbase0.94.2 to retain deleted data
Date Tue, 23 Oct 2012 22:01:59 GMT
The comment clear things up. Configuration through Java code is also working too. Thanks.
Yun

On Oct 22, 2012, at 12:34 AM, lars hofhansl <lhofhansl@yahoo.com> wrote:

> There currently is not. You have setup your column families to support this (Note that
you only need to do that once per column family!).
> This was done for flexibility, because in many cases only some of the tables need to
retain deleted cells.
> 
> These blog posts might be helpful:
> http://hadoop-hbase.blogspot.com/2011/12/deletion-in-hbase.html
> http://hadoop-hbase.blogspot.com/2011/12/hbase-data-rentention-options.html
> 
> -- Lars
> 
> 
> 
> 
> ----- Original Message -----
> From: yun peng <pengyunmomo@gmail.com>
> To: user@hbase.apache.org
> Cc: 
> Sent: Sunday, October 21, 2012 5:20 PM
> Subject: Re: How to config hbase0.94.2 to retain deleted data
> 
> Hi, Lars, that is great point. It works if I update the
> tabledescriptor with the table disabled. It looks online updating
> table descriptor generally not working...
> 
> Besides, in addition to the java API, is there any xml knob (in
> hbase0942) that can config keepdeletedcells?
> 
> Thanks a lot.
> Yun
> 
> On Sun, Oct 21, 2012 at 7:04 PM, lars hofhansl <lhofhansl@yahoo.com> wrote:
>> Not sure that you can change the Table or Column Descriptors this way through a coprocessor.
>> Did you try to create (or alter) the table such that keepDeleteCells is true:
>> 
>> hbase(main):026:0> create 'usertable', {NAME=>'cf', KEEP_DELETED_CELLS=>true}
>> 0 row(s) in 1.1660 seconds
>> 
>> hbase(main):027:0> put 'usertable', "key1", 'cf:c1', "v1", 99
>> 0 row(s) in 0.0320 seconds
>> 
>> hbase(main):028:0> delete 'usertable', "key1", 'cf:c1', 100
>> 0 row(s) in 0.0050 seconds
>> 
>> hbase(main):029:0> get 'usertable', 'key1', {COLUMN => 'cf:c1', TIMESTAMP=>
99, VERSIONS => 4}
>> COLUMN                CELL
>>   cf:c1                timestamp=99, value=v1
>> 1 row(s) in 0.0150 seconds
>> 
>> Let me know how this works for you (generally). This is a new feature I added to
0.94 to support true time-range queries.
>> 
>> -- Lars
>> 
>> 
>> ----- Original Message -----
>> From: yun peng <pengyunmomo@gmail.com>
>> To: user@hbase.apache.org
>> Cc:
>> Sent: Sunday, October 21, 2012 1:53 PM
>> Subject: How to config hbase0.94.2 to retain deleted data
>> 
>> Hi, All,
>> I want to retain all deleted key-value pairs in hbase. I have tried to
>> config HColumnDescript as follow to make it return deleted.
>> 
>>    public void postOpen(ObserverContext<RegionCoprocessorEnvironment> e) {
>>      HTableDescriptor htd = e.getEnvironment().getRegion().getTableDesc();
>>      HColumnDescriptor hcd = htd.getFamily(Bytes.toBytes("cf"));
>>      hcd.setKeepDeletedCells(true);
>>      hcd.setBlockCacheEnabled(false);
>>    }
>> 
>> However, it does not work for me, as when I issued a delete and then query
>> by an older timestamp, the old data does not show up.
>> 
>> hbase(main):119:0> put 'usertable', "key1", 'cf:c1', "v1", 99
>> hbase(main):120:0> put 'usertable', "key1", 'cf:c1', "v2", 101
>> hbase(main):121:0> delete 'usertable', "key1", 'cf:c1', 100
>> hbase(main):122:0> get 'usertable', 'key1', {COLUMN => 'cf:c1', TIMESTAMP
>> => 99, VERSIONS => 4}
>> COLUMN                CELL
>> 
>> 0 row(s) in 0.0040 seconds
>> 
>> hbase(main):123:0> get 'usertable', 'key1', {COLUMN => 'cf:c1', TIMESTAMP
>> => 100, VERSIONS => 4}
>> COLUMN                CELL
>> 
>> 0 row(s) in 0.0050 seconds
>> 
>> hbase(main):124:0> get 'usertable', 'key1', {COLUMN => 'cf:c1', TIMESTAMP
>> => 101, VERSIONS => 4}
>> COLUMN                CELL
>> 
>> cf:c1                timestamp=101, value=v2
>> 
>> 1 row(s) in 0.0050 seconds
>> 
>> Note this is a new feature in 0.94.2
>> (HBASE-4536<https://issues.apache.org/jira/browse/HBASE-4536>),
>> I did not find too many sample code online, so... any one here has
>> experience in using HBASE-4536. How should one config
>> hbase to enable this feature in hbase?
>> 
>> Thanks
>> Yun
> 

Mime
View raw message