Index: hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java =================================================================== --- hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java (revision 1470801) +++ hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java (working copy) @@ -2229,7 +2229,21 @@ } /** - * Similar to {@link #createLastOnRow(byte[], int, int, byte[], int, int, + * Similar to {@link #createLastOnRow(byte[], int, int, byte[], int, int, byte[], int, int)} + * but create the last key on the row of this KV + * (the value part of the returned KV is always empty). Used in creating + * "fake keys" to skip the row we already know is not in the file. + * @return the last key on the row of the given key-value pair + */ + public KeyValue createLastOnRow() { + return new KeyValue( + bytes, getRowOffset(), getRowLength(), + null, 0, 0, + null, 0, 0, + HConstants.LATEST_TIMESTAMP, Type.Minimum, null, 0, 0); + } + + /** * Similar to {@link #createLastOnRow(byte[], int, int, byte[], int, int, * byte[], int, int)} but creates the last key on the row/column of this KV * (the value part of the returned KV is always empty). Used in creating * "fake keys" for the multi-column Bloom filter optimization to skip the Index: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java (revision 1470801) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ScanQueryMatcher.java (working copy) @@ -281,6 +281,16 @@ int qualLength = keyLength + KeyValue.ROW_OFFSET - (offset - initialOffset) - KeyValue.TIMESTAMP_TYPE_SIZE; + byte type = kv.getType(); + // Bloom Filter optimization. See StoreFileScanner.requestSeek(...). + if (type == KeyValue.Type.Minimum.getCode()) { + if (familyLength == 0) { + return MatchCode.SEEK_NEXT_ROW; + } else { + return columns.getNextRowOrNextColumn(bytes, offset, qualLength); + } + } + long timestamp = kv.getTimestamp(); // check for early out based on timestamp alone if (columns.isDone(timestamp)) { @@ -300,7 +310,6 @@ * 7. Delete marker need to be version counted together with puts * they affect */ - byte type = kv.getType(); if (kv.isDelete()) { if (!keepDeletedCells) { // first ignore delete markers if the scanner can do so, and the Index: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java (revision 1470801) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java (working copy) @@ -1268,22 +1268,23 @@ } byte[] key; + int keyOffset, keyLen; switch (bloomFilterType) { case ROW: if (col != null) { throw new RuntimeException("Row-only Bloom filter called with " + "column specified"); } - if (rowOffset != 0 || rowLen != row.length) { - throw new AssertionError("For row-only Bloom filters the row " - + "must occupy the whole array"); - } + keyOffset = rowOffset; + keyLen = rowLen; key = row; break; case ROWCOL: key = bloomFilter.createBloomKey(row, rowOffset, rowLen, col, colOffset, colLen); + keyOffset = 0; + keyLen = key.length; break; default: @@ -1313,7 +1314,8 @@ // from the file info. For row-column Bloom filters this is not yet // a sufficient condition to return false. boolean keyIsAfterLast = lastBloomKey != null - && bloomFilter.getComparator().compare(key, lastBloomKey) > 0; + && bloomFilter.getComparator().compare(key, keyOffset, keyLen, + lastBloomKey, 0, lastBloomKey.length) > 0; if (bloomFilterType == BloomType.ROWCOL) { // Since a Row Delete is essentially a DeleteFamily applied to all @@ -1335,7 +1337,7 @@ } } else { exists = !keyIsAfterLast - && bloomFilter.contains(key, 0, key.length, bloom); + && bloomFilter.contains(key, keyOffset, keyLen, bloom); } return exists; Index: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java (revision 1470801) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFileScanner.java (working copy) @@ -285,12 +285,17 @@ } boolean haveToSeek = true; + boolean skipRow = false; if (useBloom) { - // check ROWCOL Bloom filter first. - if (reader.getBloomFilterType() == BloomType.ROWCOL) { + // check ROW or ROWCOL Bloom filter first. + if (forward && reader.getBloomFilterType() == BloomType.ROW && !kv.isDeleteFamily()) { + haveToSeek = reader.passesGeneralBloomFilter(kv.getRowArray(), + kv.getRowOffset(), kv.getRowLength(), null, 0, 0); + skipRow = !haveToSeek; + } else if (reader.getBloomFilterType() == BloomType.ROWCOL) { haveToSeek = reader.passesGeneralBloomFilter(kv.getBuffer(), - kv.getRowOffset(), kv.getRowLength(), kv.getBuffer(), - kv.getQualifierOffset(), kv.getQualifierLength()); + kv.getRowOffset(), kv.getRowLength(), kv.getBuffer(), + kv.getQualifierOffset(), kv.getQualifierLength()); } else if (this.matcher != null && !matcher.hasNullColumnInQuery() && kv.isDeleteFamily()) { // if there is no such delete family kv in the store file, @@ -334,7 +339,7 @@ // key/value and the store scanner will progress to the next column. This // is obviously not a "real real" seek, but unlike the fake KV earlier in // this method, we want this to be propagated to ScanQueryMatcher. - cur = kv.createLastOnRowCol(); + cur = skipRow ? kv.createLastOnRow() : kv.createLastOnRowCol(); realSeekDone = true; return true; Index: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java (revision 1470801) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreScanner.java (working copy) @@ -62,7 +62,6 @@ protected boolean closing = false; protected final boolean isGet; protected final boolean explicitColumnQuery; - protected final boolean useRowColBloom; /** * A flag that enables StoreFileScanner parallel-seeking */ @@ -104,11 +103,6 @@ oldestUnexpiredTS = EnvironmentEdgeManager.currentTimeMillis() - ttl; this.minVersions = minVersions; - // We look up row-column Bloom filters for multi-column queries as part of - // the seek operation. However, we also look the row-column Bloom filter - // for multi-row (non-"get") scans because this is not done in - // StoreFile.passesBloomFilter(Scan, SortedSet). - useRowColBloom = numCol > 1 || (!isGet && numCol == 1); // The parallel-seeking is on : // 1) the config value is *true* // 2) store has more than one store file @@ -591,7 +585,7 @@ //stack is reset if needed. checkReseek(); if (explicitColumnQuery && lazySeekEnabledGlobally) { - return heap.requestSeek(kv, true, useRowColBloom); + return heap.requestSeek(kv, true, true); } return heap.reseek(kv); }