Thanks for the test Evert. I'd suggest you add it to hbase-1485 as an
attachment so it gets included in final patch.
St.Ack

On Mon, Sep 6, 2010 at 3:05 AM, Evert Arckens <[email protected]> wrote:
> Here's a unittest demonstrating the use case :
>
> public class TwoCellUpdatesTest {
>   �...@test
>    public void testCellUpdates() throws Exception {
>        Configuration configuration = HBaseConfiguration.create();
>        HBaseTestingUtility hBaseTestingUtility = new
> HBaseTestingUtility(configuration);
>        hBaseTestingUtility.startMiniCluster(1);
>
>        HBaseAdmin admin = new HBaseAdmin(configuration);
>        HTableDescriptor tableDescriptor = new
> HTableDescriptor("TestTable");
>        byte[] CF = Bytes.toBytes("CF");
>        tableDescriptor.addFamily(new HColumnDescriptor(CF,
> HConstants.ALL_VERSIONS, "none", false, true, HConstants.FOREVER,
> HColumnDescriptor.DEFAULT_BLOOMFILTER));
>        admin.createTable(tableDescriptor);
>        LocalHTable hTable = new LocalHTable(configuration, "TestTable");
>
>        // Write a column with values at timestamp 1, 2 and 3
>        byte[] row = Bytes.toBytes("myRow");
>        byte[] qualifier = Bytes.toBytes("myCol");
>        Put put = new Put(row);
>        put.add(CF, qualifier, 1L, Bytes.toBytes("AAA"));
>        hTable.put(put);
>
>        put = new Put(row);
>        put.add(CF, qualifier, 2L, Bytes.toBytes("BBB"));
>        hTable.put(put);
>
>        put = new Put(row);
>        put.add(CF, qualifier, 3L, Bytes.toBytes("EEE"));
>        hTable.put(put);
>
>        Get get = new Get(row);
>        get.addColumn(CF, qualifier);
>        get.setMaxVersions();
>
>        // Check that the column indeed has the right values at timestamps 1
> and 2
>        Result result = hTable.get(get);
>        NavigableMap<Long,byte[]> navigableMap =
> result.getMap().get(CF).get(qualifier);
>        assertEquals("AAA", Bytes.toString(navigableMap.get(1L)));
>        assertEquals("BBB", Bytes.toString(navigableMap.get(2L)));
>
>        // Let major compaction do its work
>        admin.flush("TestTable");
>        admin.majorCompact("TestTable");
>        Thread.sleep(6000);
>
>        // Update the value at timestamp 1
>        put = new Put(row);
>        put.add(CF, qualifier, 1L, Bytes.toBytes("CCC"));
>        hTable.put(put);
>
> // Triggering a major compaction here would make the test succeed
> //        admin.flush("TestTable");
> //        admin.majorCompact("TestTable");
> //        Thread.sleep(6000);
>
>        // Update the value at timestamp 2
>        put = new Put(row);
>        put.add(CF, qualifier, 2L, Bytes.toBytes("DDD"));
>        hTable.put(put);
>
>        // Let major compaction do its work
>        admin.flush("TestTable");
>        admin.majorCompact("TestTable");
>        Thread.sleep(6000);
>
>        // Check that the values at timestamp 2 and 1 got updated
>        result = hTable.get(get);
>        navigableMap = result.getMap().get(CF).get(qualifier);
>        assertEquals("DDD", Bytes.toString(navigableMap.get(2L)));
>        assertEquals("CCC", Bytes.toString(navigableMap.get(1L)));
>    }
> }
>
> On Fri, Aug 20, 2010 at 7:46 PM, Jean-Daniel Cryans 
> <[email protected]>wrote:
>
>> I think that this is stretching the usage of timestamps... but would
>> you be able to write a small unit test that demonstrates it? Also did
>> you try it on both 0.20 and 0.89?
>>
>> Thx,
>>
>> J-D
>>
>> On Fri, Aug 20, 2010 at 6:18 AM, Evert Arckens <[email protected]>
>> wrote:
>> > Hi,
>> >
>> > I have a scenario of which I'm not sure if it is just another example of
>> > issue HBASE-1485, or if it is another but related issue :
>> >
>> > Write 3 cells of a column, at timestamp 1, 2 and 3.
>> > Then update the cells at timestamp 1 and 2 (more or less at the same
>> > moment).
>> >
>> > Result is that the update of the cell at timestamp 1 gets lost.
>> > If I would perform a major compaction between both updates, the update on
>> > timestamp 1 wouldn't get lost.
>> >
>> > The reason I'm not sure if it is exactly the same issue as HBASE-1485 is
>> > because that one just talks about updates on 1 cell, not on a combination
>> > updates on 2 cells at different timestamps.
>> >
>> > Regards,
>> > Evert Arckens.
>> >
>>
>

Reply via email to