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

This commit is contained in:
ramkrishna 2015-04-30 15:14:41 +05:30
parent 558cac91d0
commit 3a9c2b0c55
2 changed files with 52 additions and 5 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

@ -36,6 +36,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.KeyValue.KVComparator;
import org.apache.hadoop.hbase.fs.HFileSystem;
import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
@ -823,12 +824,21 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
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 {
NoTagsKeyValue ret = new NoTagsKeyValue(blockBuffer.array(),
blockBuffer.arrayOffset() + blockBuffer.position(), getCellBufSize());
if (this.reader.shouldIncludeMemstoreTS()) {
ret.setSequenceId(currMemstoreTS);
}
return ret;
}
}
@Override