HBASE-12078 Missing Data when scanning using PREFIX_TREE DATA-BLOCK-ENCODING
Signed-off-by: Andrew Purtell <apurtell@apache.org>
This commit is contained in:
parent
9f33edab0e
commit
947f09a62b
|
@ -93,7 +93,7 @@ public class PrefixTreeArraySearcher extends PrefixTreeArrayReversibleScanner im
|
||||||
byte searchForByte = CellUtil.getRowByte(key, currentNodeDepth);
|
byte searchForByte = CellUtil.getRowByte(key, currentNodeDepth);
|
||||||
fanIndex = currentRowNode.whichFanNode(searchForByte);
|
fanIndex = currentRowNode.whichFanNode(searchForByte);
|
||||||
if(fanIndex < 0){//no matching row. return early
|
if(fanIndex < 0){//no matching row. return early
|
||||||
int insertionPoint = -fanIndex;
|
int insertionPoint = -fanIndex - 1;
|
||||||
return fixRowFanMissReverse(insertionPoint);
|
return fixRowFanMissReverse(insertionPoint);
|
||||||
}
|
}
|
||||||
//found a match, so dig deeper into the tree
|
//found a match, so dig deeper into the tree
|
||||||
|
@ -142,7 +142,7 @@ public class PrefixTreeArraySearcher extends PrefixTreeArrayReversibleScanner im
|
||||||
byte searchForByte = CellUtil.getRowByte(key, currentNodeDepth);
|
byte searchForByte = CellUtil.getRowByte(key, currentNodeDepth);
|
||||||
fanIndex = currentRowNode.whichFanNode(searchForByte);
|
fanIndex = currentRowNode.whichFanNode(searchForByte);
|
||||||
if(fanIndex < 0){//no matching row. return early
|
if(fanIndex < 0){//no matching row. return early
|
||||||
int insertionPoint = -fanIndex;
|
int insertionPoint = -fanIndex - 1;
|
||||||
return fixRowFanMissForward(insertionPoint);
|
return fixRowFanMissForward(insertionPoint);
|
||||||
}
|
}
|
||||||
//found a match, so dig deeper into the tree
|
//found a match, so dig deeper into the tree
|
||||||
|
@ -293,6 +293,9 @@ public class PrefixTreeArraySearcher extends PrefixTreeArrayReversibleScanner im
|
||||||
}
|
}
|
||||||
return UnsignedBytes.compare(keyByte, thisByte);
|
return UnsignedBytes.compare(keyByte, thisByte);
|
||||||
}
|
}
|
||||||
|
if (!currentRowNode.hasOccurrences() && rowLength >= key.getRowLength()) { // key was shorter
|
||||||
|
return -1;
|
||||||
|
}
|
||||||
return 0;
|
return 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -366,6 +369,10 @@ public class PrefixTreeArraySearcher extends PrefixTreeArrayReversibleScanner im
|
||||||
|
|
||||||
protected CellScannerPosition fixRowFanMissReverse(int fanInsertionPoint){
|
protected CellScannerPosition fixRowFanMissReverse(int fanInsertionPoint){
|
||||||
if(fanInsertionPoint == 0){//we need to back up a row
|
if(fanInsertionPoint == 0){//we need to back up a row
|
||||||
|
if (currentRowNode.hasOccurrences()) {
|
||||||
|
populateLastNonRowFields();
|
||||||
|
return CellScannerPosition.BEFORE;
|
||||||
|
}
|
||||||
boolean foundPreviousRow = previousRow(true);//true -> position on last cell in row
|
boolean foundPreviousRow = previousRow(true);//true -> position on last cell in row
|
||||||
if(foundPreviousRow){
|
if(foundPreviousRow){
|
||||||
populateLastNonRowFields();
|
populateLastNonRowFields();
|
||||||
|
|
|
@ -232,7 +232,7 @@ public class RowNodeReader {
|
||||||
if (fanIndexInBlock >= 0) {// found it, but need to adjust for position of fan in overall block
|
if (fanIndexInBlock >= 0) {// found it, but need to adjust for position of fan in overall block
|
||||||
return fanIndexInBlock - fanOffset;
|
return fanIndexInBlock - fanOffset;
|
||||||
}
|
}
|
||||||
return fanIndexInBlock + fanOffset + 1;// didn't find it, so compensate in reverse
|
return fanIndexInBlock + fanOffset;// didn't find it, so compensate in reverse
|
||||||
}
|
}
|
||||||
|
|
||||||
public void resetFanIndex() {
|
public void resetFanIndex() {
|
||||||
|
|
|
@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.codec.prefixtree.row;
|
||||||
import java.io.ByteArrayOutputStream;
|
import java.io.ByteArrayOutputStream;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
|
import java.util.Arrays;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
|
@ -31,6 +32,7 @@ import org.apache.hadoop.hbase.KeyValueUtil;
|
||||||
import org.apache.hadoop.hbase.SmallTests;
|
import org.apache.hadoop.hbase.SmallTests;
|
||||||
import org.apache.hadoop.hbase.codec.prefixtree.decode.DecoderFactory;
|
import org.apache.hadoop.hbase.codec.prefixtree.decode.DecoderFactory;
|
||||||
import org.apache.hadoop.hbase.codec.prefixtree.encode.PrefixTreeEncoder;
|
import org.apache.hadoop.hbase.codec.prefixtree.encode.PrefixTreeEncoder;
|
||||||
|
import org.apache.hadoop.hbase.codec.prefixtree.row.data.TestRowDataSearchWithPrefix;
|
||||||
import org.apache.hadoop.hbase.codec.prefixtree.scanner.CellScannerPosition;
|
import org.apache.hadoop.hbase.codec.prefixtree.scanner.CellScannerPosition;
|
||||||
import org.apache.hadoop.hbase.codec.prefixtree.scanner.CellSearcher;
|
import org.apache.hadoop.hbase.codec.prefixtree.scanner.CellSearcher;
|
||||||
import org.apache.hadoop.hbase.util.CollectionUtils;
|
import org.apache.hadoop.hbase.util.CollectionUtils;
|
||||||
|
@ -67,7 +69,6 @@ public class TestPrefixTreeSearcher {
|
||||||
this.block = ByteBuffer.wrap(outputBytes);
|
this.block = ByteBuffer.wrap(outputBytes);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testScanForwards() throws IOException {
|
public void testScanForwards() throws IOException {
|
||||||
CellSearcher searcher = null;
|
CellSearcher searcher = null;
|
||||||
|
@ -196,4 +197,25 @@ public class TestPrefixTreeSearcher {
|
||||||
DecoderFactory.checkIn(searcher);
|
DecoderFactory.checkIn(searcher);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSeekWithPrefix() throws IOException {
|
||||||
|
if (!(rows instanceof TestRowDataSearchWithPrefix)) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
CellSearcher searcher = null;
|
||||||
|
try {
|
||||||
|
searcher = DecoderFactory.checkOut(block, true);
|
||||||
|
// seek with half bytes of second row key, should return second row
|
||||||
|
KeyValue kv = rows.getInputs().get(1);
|
||||||
|
KeyValue firstKVOnRow = KeyValueUtil.createFirstOnRow(Arrays.copyOfRange(
|
||||||
|
kv.getRowArray(), kv.getRowOffset(),
|
||||||
|
kv.getRowOffset() + kv.getRowLength() / 2));
|
||||||
|
CellScannerPosition position = searcher.positionAtOrAfter(firstKVOnRow);
|
||||||
|
Assert.assertEquals(CellScannerPosition.AFTER, position);
|
||||||
|
Assert.assertEquals(kv, searcher.current());
|
||||||
|
} finally {
|
||||||
|
DecoderFactory.checkIn(searcher);
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.codec.prefixtree.row.data.TestRowDataNumberString
|
||||||
import org.apache.hadoop.hbase.codec.prefixtree.row.data.TestRowDataQualifierByteOrdering;
|
import org.apache.hadoop.hbase.codec.prefixtree.row.data.TestRowDataQualifierByteOrdering;
|
||||||
import org.apache.hadoop.hbase.codec.prefixtree.row.data.TestRowDataRandomKeyValues;
|
import org.apache.hadoop.hbase.codec.prefixtree.row.data.TestRowDataRandomKeyValues;
|
||||||
import org.apache.hadoop.hbase.codec.prefixtree.row.data.TestRowDataRandomKeyValuesWithTags;
|
import org.apache.hadoop.hbase.codec.prefixtree.row.data.TestRowDataRandomKeyValuesWithTags;
|
||||||
|
import org.apache.hadoop.hbase.codec.prefixtree.row.data.TestRowDataSearchWithPrefix;
|
||||||
import org.apache.hadoop.hbase.codec.prefixtree.row.data.TestRowDataSearcherRowMiss;
|
import org.apache.hadoop.hbase.codec.prefixtree.row.data.TestRowDataSearcherRowMiss;
|
||||||
import org.apache.hadoop.hbase.codec.prefixtree.row.data.TestRowDataSimple;
|
import org.apache.hadoop.hbase.codec.prefixtree.row.data.TestRowDataSimple;
|
||||||
import org.apache.hadoop.hbase.codec.prefixtree.row.data.TestRowDataSingleQualifier;
|
import org.apache.hadoop.hbase.codec.prefixtree.row.data.TestRowDataSingleQualifier;
|
||||||
|
@ -87,6 +88,9 @@ public interface TestRowData {
|
||||||
all.add(new TestRowDataExerciseFInts());
|
all.add(new TestRowDataExerciseFInts());
|
||||||
all.add(new TestRowDataRandomKeyValues());
|
all.add(new TestRowDataRandomKeyValues());
|
||||||
all.add(new TestRowDataRandomKeyValuesWithTags());
|
all.add(new TestRowDataRandomKeyValuesWithTags());
|
||||||
|
|
||||||
|
//test data for HBase-12078
|
||||||
|
all.add(new TestRowDataSearchWithPrefix());
|
||||||
return all;
|
return all;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -0,0 +1,74 @@
|
||||||
|
/**
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.hadoop.hbase.codec.prefixtree.row.data;
|
||||||
|
|
||||||
|
import java.io.ByteArrayOutputStream;
|
||||||
|
import java.io.DataOutputStream;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
import org.apache.hadoop.hbase.KeyValue;
|
||||||
|
import org.apache.hadoop.hbase.codec.prefixtree.row.BaseTestRowData;
|
||||||
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
|
||||||
|
import com.google.common.collect.Lists;
|
||||||
|
|
||||||
|
public class TestRowDataSearchWithPrefix extends BaseTestRowData {
|
||||||
|
|
||||||
|
static byte[] cf = Bytes.toBytes("cf");
|
||||||
|
|
||||||
|
static byte[] cq = Bytes.toBytes("cq");
|
||||||
|
|
||||||
|
static byte[] v = Bytes.toBytes("v");
|
||||||
|
|
||||||
|
static List<KeyValue> d = Lists.newArrayList();
|
||||||
|
|
||||||
|
static long ts = 55L;
|
||||||
|
|
||||||
|
static byte[] createRowKey(int keyPart1, int keyPart2) {
|
||||||
|
ByteArrayOutputStream bos = new ByteArrayOutputStream(16);
|
||||||
|
DataOutputStream dos = new DataOutputStream(bos);
|
||||||
|
try {
|
||||||
|
dos.writeInt(keyPart1);
|
||||||
|
dos.writeInt(keyPart2);
|
||||||
|
} catch (IOException e) {
|
||||||
|
// should not happen
|
||||||
|
throw new RuntimeException(e);
|
||||||
|
}
|
||||||
|
|
||||||
|
return bos.toByteArray();
|
||||||
|
}
|
||||||
|
|
||||||
|
static {
|
||||||
|
d.add(new KeyValue(createRowKey(1, 12345), cf, cq, ts, v));
|
||||||
|
d.add(new KeyValue(createRowKey(12345, 0x01000000), cf, cq, ts, v));
|
||||||
|
d.add(new KeyValue(createRowKey(12345, 0x01010000), cf, cq, ts, v));
|
||||||
|
d.add(new KeyValue(createRowKey(12345, 0x02000000), cf, cq, ts, v));
|
||||||
|
d.add(new KeyValue(createRowKey(12345, 0x02020000), cf, cq, ts, v));
|
||||||
|
d.add(new KeyValue(createRowKey(12345, 0x03000000), cf, cq, ts, v));
|
||||||
|
d.add(new KeyValue(createRowKey(12345, 0x03030000), cf, cq, ts, v));
|
||||||
|
d.add(new KeyValue(createRowKey(12345, 0x04000000), cf, cq, ts, v));
|
||||||
|
d.add(new KeyValue(createRowKey(12345, 0x04040000), cf, cq, ts, v));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<KeyValue> getInputs() {
|
||||||
|
return d;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
Loading…
Reference in New Issue