In 0.94, from HRegion#append(): // Actually write to Memstore now for (Map.Entry<Store, List<KeyValue>> entry : tempMemstore.entrySet()) { Store store = entry.getKey(); size += store.upsert(entry.getValue()); allKVs.addAll(entry.getValue()); }
Looking at MemStore.upsert(): * Inserts the specified KeyValue into MemStore and deletes any existing * versions of the same row/family/qualifier as the specified KeyValue. I would say that for 0.94, the behavior is expected. For 0.95 / trunk, there is more freedom: upsert behavior is kept if VERSIONS for this CF == 1 Otherwise the older versions are kept around. Cheers On Mon, Jul 22, 2013 at 5:47 PM, Vladimir Rodionov <vrodio...@carrieriq.com>wrote: > Ted, It is not still clear to me. What I am observing in my tests can be > explained only if on each Append > we read the latest version of a cell, then put delete marker then put new > version. > > The question still remains: expected behavior or a bug? > > Best regards, > Vladimir Rodionov > Principal Platform Engineer > Carrier IQ, www.carrieriq.com > e-mail: vrodio...@carrieriq.com > > ________________________________________ > From: Ted Yu [yuzhih...@gmail.com] > Sent: Monday, July 22, 2013 4:54 PM > To: dev@hbase.apache.org > Subject: Re: does append delete all cell versions except the last one? > > Take a look at the release notes for: > https://issues.apache.org/jira/browse/HBASE-4583 > > The fix went to 0.95.0 > > Cheers > > On Mon, Jul 22, 2013 at 4:20 PM, Vladimir Rodionov > <vrodio...@carrieriq.com>wrote: > > > Forgot to mention: 0.94.6.1 (CDH 4.3) > > > > Best regards, > > Vladimir Rodionov > > Principal Platform Engineer > > Carrier IQ, www.carrieriq.com > > e-mail: vrodio...@carrieriq.com > > > > ________________________________________ > > From: Vladimir Rodionov > > Sent: Monday, July 22, 2013 4:17 PM > > To: dev@hbase.apache.org; lars hofhansl > > Subject: does append delete all cell versions except the last one? > > > > I am doing some tests of append operation (HTable.append(Append append)) > > and it seems that > > if cell being appended has multiple versions , all except the last one > get > > deleted. Is it correct behavior or I am doing something wrong in my > tests? > > > > The test is simple. I have a row with 3 CF and 3 qualifiers in each CF > and > > 10 versions (total # KVs stored for row is 90). I test append on the row > > (for each CF:Qualifier combination) > > > > public void testAppend() > > { > > int index = 2; > > > > LOG.error("Test append started. Testing "+index+" row"); > > > > byte[] row = data.get(index).get(0).getRow(); > > byte[] toAppend = "APPEND".getBytes(); > > LOG.error(" Append row: "+ new String(row)); > > > > // Get all data for the row > > Get get = createGet(row, null, null, null); > > get.setMaxVersions(Integer.MAX_VALUE); > > Result result = tableA.get(get); > > List<KeyValue> ll = result.list(); > > // Correct list size is 3 (total families)*3 (total > > columns) *10 (# versions) = 90 > > > > assertEquals( 90, ll.size() ); > > > > // Do append to All 3 * 3 = 9 cells in a row > > Append append = createAppend(row, > Arrays.asList(FAMILIES), > > Arrays.asList(COLUMNS), toAppend); > > Result r = tableA.append(append); > > > > ll = r.list(); > > assertEquals( 9, ll.size() ); > > > > // Check row again > > get = createGet(row, null, null, null); > > get.setMaxVersions(Integer.MAX_VALUE); > > > > result = tableA.get(get); > > > > assertEquals (90, result.size()); // This assertion > FAILS: > > actual size is 9 > > > > } > > > > Best regards, > > Vladimir Rodionov > > Principal Platform Engineer > > Carrier IQ, www.carrieriq.com > > e-mail: vrodio...@carrieriq.com > > > > > > 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 notificati...@carrieriq.com and > > delete or destroy any copy of this message and its attachments. > > > > 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 notificati...@carrieriq.com and > delete or destroy any copy of this message and its attachments. >