Skip to content

Commit 987012d

Browse files
mwkangApache9
authored andcommitted
HBASE-29254 StoreScanner returns incorrect row after flush due to topChanged behavior (#6900)
Signed-off-by: Duo Zhang <zhangduo@apache.org> (cherry picked from commit 99bd5b5)
1 parent 30cae7a commit 987012d

2 files changed

Lines changed: 102 additions & 6 deletions

File tree

hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java

Lines changed: 5 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -750,15 +750,15 @@ public boolean next(List<? super ExtendedCell> outResult, ScannerContext scanner
750750
}
751751
matcher.clearCurrentRow();
752752
seekOrSkipToNextRow(cell);
753-
NextState stateAfterSeekNextRow = needToReturn(outResult);
753+
NextState stateAfterSeekNextRow = needToReturn();
754754
if (stateAfterSeekNextRow != null) {
755755
return scannerContext.setScannerState(stateAfterSeekNextRow).hasMoreValues();
756756
}
757757
break;
758758

759759
case SEEK_NEXT_COL:
760760
seekOrSkipToNextColumn(cell);
761-
NextState stateAfterSeekNextColumn = needToReturn(outResult);
761+
NextState stateAfterSeekNextColumn = needToReturn();
762762
if (stateAfterSeekNextColumn != null) {
763763
return scannerContext.setScannerState(stateAfterSeekNextColumn).hasMoreValues();
764764
}
@@ -776,7 +776,7 @@ public boolean next(List<? super ExtendedCell> outResult, ScannerContext scanner
776776
((!scan.isReversed() && difference > 0) || (scan.isReversed() && difference < 0))
777777
) {
778778
seekAsDirection(nextKV);
779-
NextState stateAfterSeekByHint = needToReturn(outResult);
779+
NextState stateAfterSeekByHint = needToReturn();
780780
if (stateAfterSeekByHint != null) {
781781
return scannerContext.setScannerState(stateAfterSeekByHint).hasMoreValues();
782782
}
@@ -833,11 +833,10 @@ private void updateMetricsStore(boolean memstoreRead) {
833833
* memstore scanner is replaced by hfile scanner after #reopenAfterFlush. If the row of top cell
834834
* is changed, we should return the current cells. Otherwise, we may return the cells across
835835
* different rows.
836-
* @param outResult the cells which are visible for user scan
837836
* @return null is the top cell doesn't change. Otherwise, the NextState to return
838837
*/
839-
private NextState needToReturn(List<? super ExtendedCell> outResult) {
840-
if (!outResult.isEmpty() && topChanged) {
838+
private NextState needToReturn() {
839+
if (topChanged) {
841840
return heap.peek() == null ? NextState.NO_MORE_VALUES : NextState.MORE_VALUES;
842841
}
843842
return null;

hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java

Lines changed: 97 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -67,6 +67,7 @@
6767
import java.util.concurrent.locks.ReentrantReadWriteLock;
6868
import java.util.function.Consumer;
6969
import java.util.function.IntBinaryOperator;
70+
import java.util.function.IntConsumer;
7071
import org.apache.hadoop.conf.Configuration;
7172
import org.apache.hadoop.fs.FSDataOutputStream;
7273
import org.apache.hadoop.fs.FileStatus;
@@ -1277,6 +1278,12 @@ private ExtendedCell createCell(byte[] row, byte[] qualifier, long ts, long sequ
12771278
.setValue(value).setSequenceId(sequenceId).build();
12781279
}
12791280

1281+
private ExtendedCell createDeleteCell(byte[] row, byte[] qualifier, long ts, long sequenceId) {
1282+
return ExtendedCellBuilderFactory.create(CellBuilderType.DEEP_COPY).setRow(row)
1283+
.setFamily(family).setQualifier(qualifier).setTimestamp(ts).setType(Cell.Type.DeleteColumn)
1284+
.setSequenceId(sequenceId).build();
1285+
}
1286+
12801287
@Test
12811288
public void testFlushBeforeCompletingScanWoFilter() throws IOException, InterruptedException {
12821289
final AtomicBoolean timeToGoNextRow = new AtomicBoolean(false);
@@ -1421,6 +1428,74 @@ public long getSmallestReadPoint(HStore store) {
14211428
}
14221429
}
14231430

1431+
@Test
1432+
public void testFlushBeforeCompletingScanWithDeleteCell() throws IOException {
1433+
final Configuration conf = HBaseConfiguration.create();
1434+
1435+
byte[] r1 = Bytes.toBytes("row1");
1436+
byte[] r2 = Bytes.toBytes("row2");
1437+
1438+
byte[] value1 = Bytes.toBytes("value1");
1439+
byte[] value2 = Bytes.toBytes("value2");
1440+
1441+
final MemStoreSizing memStoreSizing = new NonThreadSafeMemStoreSizing();
1442+
final long ts = EnvironmentEdgeManager.currentTime();
1443+
final long seqId = 100;
1444+
1445+
init(name.getMethodName(), conf, TableDescriptorBuilder.newBuilder(TableName.valueOf(table)),
1446+
ColumnFamilyDescriptorBuilder.newBuilder(family).setMaxVersions(1).build(),
1447+
new MyStoreHook() {
1448+
@Override
1449+
long getSmallestReadPoint(HStore store) {
1450+
return seqId + 3;
1451+
}
1452+
});
1453+
1454+
store.add(createCell(r1, qf1, ts + 1, seqId + 1, value2), memStoreSizing);
1455+
store.add(createCell(r1, qf2, ts + 1, seqId + 1, value2), memStoreSizing);
1456+
store.add(createCell(r1, qf3, ts + 1, seqId + 1, value2), memStoreSizing);
1457+
1458+
store.add(createDeleteCell(r1, qf1, ts + 2, seqId + 2), memStoreSizing);
1459+
store.add(createDeleteCell(r1, qf2, ts + 2, seqId + 2), memStoreSizing);
1460+
store.add(createDeleteCell(r1, qf3, ts + 2, seqId + 2), memStoreSizing);
1461+
1462+
store.add(createCell(r2, qf1, ts + 3, seqId + 3, value1), memStoreSizing);
1463+
store.add(createCell(r2, qf2, ts + 3, seqId + 3, value1), memStoreSizing);
1464+
store.add(createCell(r2, qf3, ts + 3, seqId + 3, value1), memStoreSizing);
1465+
1466+
Scan scan = new Scan().withStartRow(r1);
1467+
1468+
try (final InternalScanner scanner =
1469+
new StoreScanner(store, store.getScanInfo(), scan, null, seqId + 3) {
1470+
@Override
1471+
protected KeyValueHeap newKVHeap(List<? extends KeyValueScanner> scanners,
1472+
CellComparator comparator) throws IOException {
1473+
return new MyKeyValueHeap(scanners, comparator, recordBlockSizeCallCount -> {
1474+
if (recordBlockSizeCallCount == 6) {
1475+
try {
1476+
flushStore(store, id++);
1477+
} catch (IOException e) {
1478+
throw new RuntimeException(e);
1479+
}
1480+
}
1481+
});
1482+
}
1483+
}) {
1484+
List<Cell> cellResult = new ArrayList<>();
1485+
1486+
scanner.next(cellResult);
1487+
assertEquals(0, cellResult.size());
1488+
1489+
cellResult.clear();
1490+
1491+
scanner.next(cellResult);
1492+
assertEquals(3, cellResult.size());
1493+
for (Cell cell : cellResult) {
1494+
assertArrayEquals(r2, CellUtil.cloneRow(cell));
1495+
}
1496+
}
1497+
}
1498+
14241499
@Test
14251500
public void testCreateScannerAndSnapshotConcurrently() throws IOException, InterruptedException {
14261501
Configuration conf = HBaseConfiguration.create();
@@ -3137,6 +3212,28 @@ public List<T> subList(int fromIndex, int toIndex) {
31373212
}
31383213
}
31393214

3215+
private interface MyKeyValueHeapHook {
3216+
void onRecordBlockSize(int recordBlockSizeCallCount);
3217+
}
3218+
3219+
private static class MyKeyValueHeap extends KeyValueHeap {
3220+
private final MyKeyValueHeapHook hook;
3221+
private int recordBlockSizeCallCount;
3222+
3223+
public MyKeyValueHeap(List<? extends KeyValueScanner> scanners, CellComparator comparator,
3224+
MyKeyValueHeapHook hook) throws IOException {
3225+
super(scanners, comparator);
3226+
this.hook = hook;
3227+
}
3228+
3229+
@Override
3230+
public void recordBlockSize(IntConsumer blockSizeConsumer) {
3231+
recordBlockSizeCallCount++;
3232+
hook.onRecordBlockSize(recordBlockSizeCallCount);
3233+
super.recordBlockSize(blockSizeConsumer);
3234+
}
3235+
}
3236+
31403237
public static class MyCompactingMemStore2 extends CompactingMemStore {
31413238
private static final String LARGE_CELL_THREAD_NAME = "largeCellThread";
31423239
private static final String SMALL_CELL_THREAD_NAME = "smallCellThread";

0 commit comments

Comments
 (0)