HBASE-21640 Remove the TODO when increment zero
This commit is contained in:
parent
44dec60054
commit
4281cb3b95
|
@ -7963,8 +7963,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
|
||||
/**
|
||||
* Reckon the Cells to apply to WAL, memstore, and to return to the Client; these Sets are not
|
||||
* always the same dependent on whether to write WAL or if the amount to increment is zero (in
|
||||
* this case we write back nothing, just return latest Cell value to the client).
|
||||
* always the same dependent on whether to write WAL.
|
||||
*
|
||||
* @param results Fill in here what goes back to the Client if it is non-null (if null, client
|
||||
* doesn't want results).
|
||||
|
@ -8006,9 +8005,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
* @param op Whether Increment or Append
|
||||
* @param mutation The encompassing Mutation object
|
||||
* @param deltas Changes to apply to this Store; either increment amount or data to append
|
||||
* @param results In here we accumulate all the Cells we are to return to the client; this List
|
||||
* can be larger than what we return in case where delta is zero; i.e. don't write
|
||||
* out new values, just return current value. If null, client doesn't want results returned.
|
||||
* @param results In here we accumulate all the Cells we are to return to the client. If null,
|
||||
* client doesn't want results returned.
|
||||
* @return Resulting Cells after <code>deltas</code> have been applied to current
|
||||
* values. Side effect is our filling out of the <code>results</code> List.
|
||||
*/
|
||||
|
@ -8036,33 +8034,25 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
for (int i = 0; i < deltas.size(); i++) {
|
||||
Cell delta = deltas.get(i);
|
||||
Cell currentValue = null;
|
||||
boolean firstWrite = false;
|
||||
if (currentValuesIndex < currentValues.size() &&
|
||||
CellUtil.matchingQualifier(currentValues.get(currentValuesIndex), delta)) {
|
||||
currentValue = currentValues.get(currentValuesIndex);
|
||||
if (i < (deltas.size() - 1) && !CellUtil.matchingQualifier(delta, deltas.get(i + 1))) {
|
||||
currentValuesIndex++;
|
||||
}
|
||||
} else {
|
||||
firstWrite = true;
|
||||
}
|
||||
// Switch on whether this an increment or an append building the new Cell to apply.
|
||||
Cell newCell = null;
|
||||
MutationType mutationType = null;
|
||||
boolean apply = true;
|
||||
switch (op) {
|
||||
case INCREMENT:
|
||||
mutationType = MutationType.INCREMENT;
|
||||
// If delta amount to apply is 0, don't write WAL or MemStore.
|
||||
long deltaAmount = getLongValue(delta);
|
||||
// TODO: Does zero value mean reset Cell? For example, the ttl.
|
||||
apply = deltaAmount != 0;
|
||||
final long newValue = currentValue == null ? deltaAmount : getLongValue(currentValue) + deltaAmount;
|
||||
newCell = reckonDelta(delta, currentValue, columnFamily, now, mutation, (oldCell) -> Bytes.toBytes(newValue));
|
||||
break;
|
||||
case APPEND:
|
||||
mutationType = MutationType.APPEND;
|
||||
// Always apply Append. TODO: Does empty delta value mean reset Cell? It seems to.
|
||||
newCell = reckonDelta(delta, currentValue, columnFamily, now, mutation, (oldCell) ->
|
||||
ByteBuffer.wrap(new byte[delta.getValueLength() + oldCell.getValueLength()])
|
||||
.put(oldCell.getValueArray(), oldCell.getValueOffset(), oldCell.getValueLength())
|
||||
|
@ -8078,10 +8068,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
newCell =
|
||||
coprocessorHost.postMutationBeforeWAL(mutationType, mutation, currentValue, newCell);
|
||||
}
|
||||
// If apply, we need to update memstore/WAL with new value; add it toApply.
|
||||
if (apply || firstWrite) {
|
||||
toApply.add(newCell);
|
||||
}
|
||||
toApply.add(newCell);
|
||||
// Add to results to get returned to the Client. If null, cilent does not want results.
|
||||
if (results != null) {
|
||||
results.add(newCell);
|
||||
|
|
|
@ -208,14 +208,13 @@ public class TestDurability {
|
|||
assertEquals(1, Bytes.toLong(res.getValue(FAMILY, col1)));
|
||||
verifyWALCount(wals, wal, 2);
|
||||
|
||||
// col1: amount = 0, 0 write back to WAL
|
||||
// col1: amount = 0, 1 write back to WAL
|
||||
inc1 = new Increment(row1);
|
||||
inc1.addColumn(FAMILY, col1, 0);
|
||||
res = region.increment(inc1);
|
||||
assertEquals(1, res.size());
|
||||
assertEquals(1, Bytes.toLong(res.getValue(FAMILY, col1)));
|
||||
verifyWALCount(wals, wal, 2);
|
||||
|
||||
verifyWALCount(wals, wal, 3);
|
||||
// col1: amount = 0, col2: amount = 0, col3: amount = 0
|
||||
// 1 write back to WAL
|
||||
inc1 = new Increment(row1);
|
||||
|
@ -227,7 +226,7 @@ public class TestDurability {
|
|||
assertEquals(1, Bytes.toLong(res.getValue(FAMILY, col1)));
|
||||
assertEquals(0, Bytes.toLong(res.getValue(FAMILY, col2)));
|
||||
assertEquals(0, Bytes.toLong(res.getValue(FAMILY, col3)));
|
||||
verifyWALCount(wals, wal, 3);
|
||||
verifyWALCount(wals, wal, 4);
|
||||
|
||||
// col1: amount = 5, col2: amount = 4, col3: amount = 3
|
||||
// 1 write back to WAL
|
||||
|
@ -240,7 +239,7 @@ public class TestDurability {
|
|||
assertEquals(6, Bytes.toLong(res.getValue(FAMILY, col1)));
|
||||
assertEquals(4, Bytes.toLong(res.getValue(FAMILY, col2)));
|
||||
assertEquals(3, Bytes.toLong(res.getValue(FAMILY, col3)));
|
||||
verifyWALCount(wals, wal, 4);
|
||||
verifyWALCount(wals, wal, 5);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
Loading…
Reference in New Issue