You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by an...@apache.org on 2016/03/11 09:29:03 UTC

hbase git commit: HBASE-15180 Reduce garbage created while reading Cells from Codec Decoder.

Repository: hbase
Updated Branches:
  refs/heads/master a979d8558 -> eea8b38df


HBASE-15180 Reduce garbage created while reading Cells from Codec Decoder.


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/eea8b38d
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/eea8b38d
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/eea8b38d

Branch: refs/heads/master
Commit: eea8b38dfa0180d3e6f93d3e8055d5d4fbf673c3
Parents: a979d85
Author: anoopsjohn <an...@gmail.com>
Authored: Fri Mar 11 13:58:41 2016 +0530
Committer: anoopsjohn <an...@gmail.com>
Committed: Fri Mar 11 13:58:41 2016 +0530

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/ipc/IPCUtil.java    | 71 +++++++++++---------
 .../apache/hadoop/hbase/ipc/TestIPCUtil.java    |  2 +-
 .../apache/hadoop/hbase/codec/CellCodec.java    |  7 ++
 .../hadoop/hbase/codec/CellCodecWithTags.java   |  7 ++
 .../org/apache/hadoop/hbase/codec/Codec.java    |  2 +
 .../hadoop/hbase/codec/KeyValueCodec.java       | 39 +++++++++++
 .../hbase/codec/KeyValueCodecWithTags.java      | 19 ++++++
 .../hadoop/hbase/util/ByteBufferUtils.java      | 13 ++++
 .../apache/hadoop/hbase/codec/MessageCodec.java |  7 ++
 .../org/apache/hadoop/hbase/ipc/RpcServer.java  |  2 +-
 .../hbase/regionserver/wal/WALCellCodec.java    |  7 ++
 .../hadoop/hbase/regionserver/TestTags.java     |  6 +-
 12 files changed, 146 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/eea8b38d/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java
index 22c5cc1..d98d81d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/IPCUtil.java
@@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.ipc;
 
 import java.io.DataInput;
 import java.io.IOException;
-import java.io.InputStream;
 import java.io.OutputStream;
 import java.nio.BufferOverflowException;
 import java.nio.ByteBuffer;
@@ -178,50 +177,58 @@ public class IPCUtil {
    * @throws IOException
    */
   public CellScanner createCellScanner(final Codec codec, final CompressionCodec compressor,
-      final byte [] cellBlock)
-  throws IOException {
-    return createCellScanner(codec, compressor, ByteBuffer.wrap(cellBlock));
+      final byte[] cellBlock) throws IOException {
+    // Use this method from Client side to create the CellScanner
+    ByteBuffer cellBlockBuf = ByteBuffer.wrap(cellBlock);
+    if (compressor != null) {
+      cellBlockBuf = decompress(compressor, cellBlockBuf);
+    }
+    // Not making the Decoder over the ByteBuffer purposefully. The Decoder over the BB will
+    // make Cells directly over the passed BB. This method is called at client side and we don't
+    // want the Cells to share the same byte[] where the RPC response is being read. Caching of any
+    // of the Cells at user's app level will make it not possible to GC the response byte[]
+    return codec.getDecoder(new ByteBufferInputStream(cellBlockBuf));
   }
 
   /**
    * @param codec
    * @param cellBlock ByteBuffer containing the cells written by the Codec. The buffer should be
    * position()'ed at the start of the cell block and limit()'ed at the end.
-   * @return CellScanner to work against the content of <code>cellBlock</code>
+   * @return CellScanner to work against the content of <code>cellBlock</code>.
+   * All cells created out of the CellScanner will share the same ByteBuffer being passed.
    * @throws IOException
    */
-  public CellScanner createCellScanner(final Codec codec, final CompressionCodec compressor,
-      final ByteBuffer cellBlock)
-  throws IOException {
+  public CellScanner createCellScannerReusingBuffers(final Codec codec,
+      final CompressionCodec compressor, ByteBuffer cellBlock) throws IOException {
+    // Use this method from HRS to create the CellScanner
     // If compressed, decompress it first before passing it on else we will leak compression
     // resources if the stream is not closed properly after we let it out.
-    InputStream is = null;
     if (compressor != null) {
-      // GZIPCodec fails w/ NPE if no configuration.
-      if (compressor instanceof Configurable) ((Configurable)compressor).setConf(this.conf);
-      Decompressor poolDecompressor = CodecPool.getDecompressor(compressor);
-      CompressionInputStream cis =
-        compressor.createInputStream(new ByteBufferInputStream(cellBlock), poolDecompressor);
-      ByteBufferOutputStream bbos = null;
-      try {
-        // TODO: This is ugly.  The buffer will be resized on us if we guess wrong.
-        // TODO: Reuse buffers.
-        bbos = new ByteBufferOutputStream(cellBlock.remaining() *
-          this.cellBlockDecompressionMultiplier);
-        IOUtils.copy(cis, bbos);
-        bbos.close();
-        ByteBuffer bb = bbos.getByteBuffer();
-        is = new ByteBufferInputStream(bb);
-      } finally {
-        if (is != null) is.close();
-        if (bbos != null) bbos.close();
+      cellBlock = decompress(compressor, cellBlock);
+    }
+    return codec.getDecoder(cellBlock);
+  }
 
-        CodecPool.returnDecompressor(poolDecompressor);
-      }
-    } else {
-      is = new ByteBufferInputStream(cellBlock);
+  private ByteBuffer decompress(CompressionCodec compressor, ByteBuffer cellBlock)
+      throws IOException {
+    // GZIPCodec fails w/ NPE if no configuration.
+    if (compressor instanceof Configurable) ((Configurable) compressor).setConf(this.conf);
+    Decompressor poolDecompressor = CodecPool.getDecompressor(compressor);
+    CompressionInputStream cis = compressor.createInputStream(new ByteBufferInputStream(cellBlock),
+        poolDecompressor);
+    ByteBufferOutputStream bbos = null;
+    try {
+      // TODO: This is ugly. The buffer will be resized on us if we guess wrong.
+      // TODO: Reuse buffers.
+      bbos = new ByteBufferOutputStream(
+          cellBlock.remaining() * this.cellBlockDecompressionMultiplier);
+      IOUtils.copy(cis, bbos);
+      bbos.close();
+      cellBlock = bbos.getByteBuffer();
+    } finally {
+      CodecPool.returnDecompressor(poolDecompressor);
     }
-    return codec.getDecoder(is);
+    return cellBlock;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/eea8b38d/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestIPCUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestIPCUtil.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestIPCUtil.java
index 0038aec..c90b275 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestIPCUtil.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/ipc/TestIPCUtil.java
@@ -79,7 +79,7 @@ public class TestIPCUtil {
     CellScanner cellScanner = sized? getSizedCellScanner(cells):
       CellUtil.createCellScanner(Arrays.asList(cells).iterator());
     ByteBuffer bb = util.buildCellBlock(codec, compressor, cellScanner);
-    cellScanner = util.createCellScanner(codec, compressor, bb);
+    cellScanner = util.createCellScannerReusingBuffers(codec, compressor, bb);
     int i = 0;
     while (cellScanner.advance()) {
       i++;

http://git-wip-us.apache.org/repos/asf/hbase/blob/eea8b38d/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/CellCodec.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/CellCodec.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/CellCodec.java
index 666f440..d6b64f6 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/CellCodec.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/CellCodec.java
@@ -20,12 +20,14 @@ package org.apache.hadoop.hbase.codec;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.nio.ByteBuffer;
 
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.io.ByteBufferInputStream;
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
@@ -116,6 +118,11 @@ public class CellCodec implements Codec {
   }
 
   @Override
+  public Decoder getDecoder(ByteBuffer buf) {
+    return getDecoder(new ByteBufferInputStream(buf));
+  }
+
+  @Override
   public Encoder getEncoder(OutputStream os) {
     return new CellEncoder(os);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/eea8b38d/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/CellCodecWithTags.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/CellCodecWithTags.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/CellCodecWithTags.java
index d79be17..7326884 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/CellCodecWithTags.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/CellCodecWithTags.java
@@ -20,12 +20,14 @@ package org.apache.hadoop.hbase.codec;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.nio.ByteBuffer;
 
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.io.ByteBufferInputStream;
 import org.apache.hadoop.hbase.util.Bytes;
 
 /**
@@ -117,6 +119,11 @@ public class CellCodecWithTags implements Codec {
   }
 
   @Override
+  public Decoder getDecoder(ByteBuffer buf) {
+    return getDecoder(new ByteBufferInputStream(buf));
+  }
+
+  @Override
   public Encoder getEncoder(OutputStream os) {
     return new CellEncoder(os);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/eea8b38d/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/Codec.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/Codec.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/Codec.java
index de44ec6..c8a4cdc 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/Codec.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/Codec.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.codec;
 
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.nio.ByteBuffer;
 
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
@@ -50,5 +51,6 @@ public interface Codec {
   interface Decoder extends CellScanner {};
 
   Decoder getDecoder(InputStream is);
+  Decoder getDecoder(ByteBuffer buf);
   Encoder getEncoder(OutputStream os);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/eea8b38d/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodec.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodec.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodec.java
index f99bfcb..df2e740 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodec.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodec.java
@@ -20,11 +20,14 @@ package org.apache.hadoop.hbase.codec;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.nio.ByteBuffer;
 
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.KeyValueUtil;
+import org.apache.hadoop.hbase.NoTagsKeyValue;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.util.ByteBufferUtils;
 
 /**
  * Codec that does KeyValue version 1 serialization.
@@ -69,6 +72,37 @@ public class KeyValueCodec implements Codec {
     }
   }
 
+  public static class ByteBufferedKeyValueDecoder implements Codec.Decoder {
+
+    protected final ByteBuffer buf;
+    protected Cell current = null;
+
+    public ByteBufferedKeyValueDecoder(ByteBuffer buf) {
+      this.buf = buf;
+    }
+
+    @Override
+    public boolean advance() throws IOException {
+      if (this.buf.remaining() <= 0) {
+        return false;
+      }
+      int len = ByteBufferUtils.toInt(buf);
+      assert buf.hasArray();
+      this.current = createCell(buf.array(), buf.arrayOffset() + buf.position(), len);
+      buf.position(buf.position() + len);
+      return true;
+    }
+
+    @Override
+    public Cell current() {
+      return this.current;
+    }
+
+    protected Cell createCell(byte[] buf, int offset, int len) {
+      return new NoTagsKeyValue(buf, offset, len);
+    }
+  }
+
   /**
    * Implementation depends on {@link InputStream#available()}
    */
@@ -78,6 +112,11 @@ public class KeyValueCodec implements Codec {
   }
 
   @Override
+  public Decoder getDecoder(ByteBuffer buf) {
+    return new ByteBufferedKeyValueDecoder(buf);
+  }
+
+  @Override
   public Encoder getEncoder(OutputStream os) {
     return new KeyValueEncoder(os);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/eea8b38d/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodecWithTags.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodecWithTags.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodecWithTags.java
index ad762b4..714cc38 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodecWithTags.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/codec/KeyValueCodecWithTags.java
@@ -20,9 +20,11 @@ package org.apache.hadoop.hbase.codec;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.nio.ByteBuffer;
 
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 
@@ -75,6 +77,18 @@ public class KeyValueCodecWithTags implements Codec {
     }
   }
 
+  public static class ByteBufferedKeyValueDecoder
+      extends KeyValueCodec.ByteBufferedKeyValueDecoder {
+
+    public ByteBufferedKeyValueDecoder(ByteBuffer buf) {
+      super(buf);
+    }
+
+    protected Cell createCell(byte[] buf, int offset, int len) {
+      return new KeyValue(buf, offset, len);
+    }
+  }
+
   /**
    * Implementation depends on {@link InputStream#available()}
    */
@@ -87,4 +101,9 @@ public class KeyValueCodecWithTags implements Codec {
   public Encoder getEncoder(OutputStream os) {
     return new KeyValueEncoder(os);
   }
+
+  @Override
+  public Decoder getDecoder(ByteBuffer buf) {
+    return new ByteBufferedKeyValueDecoder(buf);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/eea8b38d/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
index 6ca97da..df23614 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ByteBufferUtils.java
@@ -782,6 +782,19 @@ public final class ByteBufferUtils {
   }
 
   /**
+   * Reads an int value at the given buffer's current position. Also advances the buffer's position
+   */
+  public static int toInt(ByteBuffer buffer) {
+    if (UNSAFE_UNALIGNED) {
+      int i = UnsafeAccess.toInt(buffer, buffer.position());
+      buffer.position(buffer.position() + Bytes.SIZEOF_INT);
+      return i;
+    } else {
+      return buffer.getInt();
+    }
+  }
+
+  /**
    * Reads an int value at the given buffer's offset.
    * @param buffer
    * @param offset

http://git-wip-us.apache.org/repos/asf/hbase/blob/eea8b38d/hbase-server/src/main/java/org/apache/hadoop/hbase/codec/MessageCodec.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/codec/MessageCodec.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/codec/MessageCodec.java
index 6c894a5..8f08539 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/codec/MessageCodec.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/codec/MessageCodec.java
@@ -20,9 +20,11 @@ package org.apache.hadoop.hbase.codec;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.nio.ByteBuffer;
 
 import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.io.ByteBufferInputStream;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
@@ -80,6 +82,11 @@ public class MessageCodec implements Codec {
   }
 
   @Override
+  public Decoder getDecoder(ByteBuffer buf) {
+    return getDecoder(new ByteBufferInputStream(buf));
+  }
+
+  @Override
   public Encoder getEncoder(OutputStream os) {
     return new MessageEncoder(os);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/eea8b38d/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
index 6ddfb9a..ef6c198 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
@@ -1884,7 +1884,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
         }
         if (header.hasCellBlockMeta()) {
           buf.position(offset);
-          cellScanner = ipcUtil.createCellScanner(this.codec, this.compressionCodec, buf);
+          cellScanner = ipcUtil.createCellScannerReusingBuffers(this.codec, this.compressionCodec, buf);
         }
       } catch (Throwable t) {
         InetSocketAddress address = getListenerAddress();

http://git-wip-us.apache.org/repos/asf/hbase/blob/eea8b38d/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
index 05929fa..6b89e89 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCellCodec.java
@@ -21,6 +21,7 @@ import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.nio.ByteBuffer;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
@@ -32,6 +33,7 @@ import org.apache.hadoop.hbase.codec.BaseDecoder;
 import org.apache.hadoop.hbase.codec.BaseEncoder;
 import org.apache.hadoop.hbase.codec.Codec;
 import org.apache.hadoop.hbase.codec.KeyValueCodecWithTags;
+import org.apache.hadoop.hbase.io.ByteBufferInputStream;
 import org.apache.hadoop.hbase.io.util.Dictionary;
 import org.apache.hadoop.hbase.io.util.StreamUtils;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -348,6 +350,11 @@ public class WALCellCodec implements Codec {
   }
 
   @Override
+  public Decoder getDecoder(ByteBuffer buf) {
+    return getDecoder(new ByteBufferInputStream(buf));
+  }
+
+  @Override
   public Encoder getEncoder(OutputStream os) {
     return (compression == null)
         ? new EnsureKvEncoder(os) : new CompressedKvEncoder(os, compression);

http://git-wip-us.apache.org/repos/asf/hbase/blob/eea8b38d/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTags.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTags.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTags.java
index d99643d..3a9ace2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTags.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTags.java
@@ -221,7 +221,8 @@ public class TestTags {
           CellScanner cellScanner = result.cellScanner();
           cellScanner.advance();
           KeyValue current = (KeyValue) cellScanner.current();
-          assertTrue(current.getValueOffset() + current.getValueLength() == current.getLength());
+          assertTrue(current.getValueOffset() + current.getValueLength() == current.getOffset()
+              + current.getLength());
         }
       } finally {
         if (scanner != null)
@@ -239,7 +240,8 @@ public class TestTags {
           CellScanner cellScanner = result.cellScanner();
           cellScanner.advance();
           KeyValue current = (KeyValue) cellScanner.current();
-          assertTrue(current.getValueOffset() + current.getValueLength() == current.getLength());
+          assertTrue(current.getValueOffset() + current.getValueLength() == current.getOffset()
+              + current.getLength());
         }
       } finally {
         if (scanner != null) {