HBASE-13123 - Minor bug in ROW bloom filter (Ram)
This commit is contained in:
parent
4980bfe642
commit
4fb6f91cba
|
@ -33,7 +33,6 @@ import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
|
@ -44,6 +43,7 @@ import org.apache.hadoop.hbase.HDFSBlocksDistribution;
|
||||||
import org.apache.hadoop.hbase.KeyValue;
|
import org.apache.hadoop.hbase.KeyValue;
|
||||||
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
||||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||||
|
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.hbase.client.Scan;
|
import org.apache.hadoop.hbase.client.Scan;
|
||||||
import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
|
import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
|
||||||
import org.apache.hadoop.hbase.io.hfile.BlockType;
|
import org.apache.hadoop.hbase.io.hfile.BlockType;
|
||||||
|
@ -1289,7 +1289,7 @@ public class StoreFile {
|
||||||
// columns, a file might be skipped if using row+col Bloom filter.
|
// columns, a file might be skipped if using row+col Bloom filter.
|
||||||
// In order to ensure this file is included an additional check is
|
// In order to ensure this file is included an additional check is
|
||||||
// required looking only for a row bloom.
|
// required looking only for a row bloom.
|
||||||
byte[] rowBloomKey = bloomFilter.createBloomKey(row, 0, row.length,
|
byte[] rowBloomKey = bloomFilter.createBloomKey(row, rowOffset, rowLen,
|
||||||
null, 0, 0);
|
null, 0, 0);
|
||||||
|
|
||||||
if (keyIsAfterLast
|
if (keyIsAfterLast
|
||||||
|
|
Loading…
Reference in New Issue