HBASE-13579 - Avoid isCellTTLExpired() for NO-TAG cases (Ram)

This commit is contained in:
ramkrishna 2015-05-06 12:10:31 +05:30
parent 308d515949
commit 426c7eef09
3 changed files with 60 additions and 1 deletions

View File

@ -0,0 +1,37 @@
/**
* Copyright The Apache Software Foundation
*
* 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;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
/**
* An extension of the KeyValue where the tags length is always 0
*/
@InterfaceAudience.Private
public class NoTagsKeyValue extends KeyValue {
public NoTagsKeyValue(byte[] bytes, int offset, int length) {
super(bytes, offset, length);
}
@Override
public int getTagsLength() {
return 0;
}
}

View File

@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValue.KVComparator;
import org.apache.hadoop.hbase.NoTagsKeyValue;
import org.apache.hadoop.hbase.fs.HFileSystem;
import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoder;
@ -747,7 +748,11 @@ public class HFileReaderV2 extends AbstractHFileReader {
if (!isSeeked())
return null;
KeyValue ret = new KeyValue(blockBuffer.array(), blockBuffer.arrayOffset()
return formNoTagsKeyValue();
}
protected Cell formNoTagsKeyValue() {
NoTagsKeyValue ret = new NoTagsKeyValue(blockBuffer.array(), blockBuffer.arrayOffset()
+ blockBuffer.position(), getCellBufSize());
if (this.reader.shouldIncludeMemstoreTS()) {
ret.setSequenceId(currMemstoreTS);

View File

@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.NoTagsKeyValue;
import org.apache.hadoop.hbase.fs.HFileSystem;
import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
import org.apache.hadoop.hbase.io.crypto.Cipher;
@ -190,6 +191,22 @@ public class HFileReaderV3 extends HFileReaderV2 {
return kvBufSize;
}
@Override
public Cell getKeyValue() {
if (!isSeeked())
return null;
if (currTagsLen > 0) {
KeyValue ret = new KeyValue(blockBuffer.array(), blockBuffer.arrayOffset()
+ blockBuffer.position(), getCellBufSize());
if (this.reader.shouldIncludeMemstoreTS()) {
ret.setSequenceId(currMemstoreTS);
}
return ret;
} else {
return formNoTagsKeyValue();
}
}
protected void setNonSeekedState() {
super.setNonSeekedState();
currTagsLen = 0;