You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2014/05/23 22:55:39 UTC

[1/4] git commit: HBASE-11227 Mention 8- and 16-bit fixed-with encodings in OrderedBytes docstring (Nick Dimiduk)

Repository: hbase
Updated Branches:
  refs/heads/0.98 b985e4fd0 -> d73711c80
  refs/heads/master 7ee058bc2 -> 8b145419e


HBASE-11227 Mention 8- and 16-bit fixed-with encodings in OrderedBytes docstring (Nick Dimiduk)


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

Branch: refs/heads/master
Commit: 30aab8b5ea667b97f7a161893cf98b82d7f34fc6
Parents: 7ee058b
Author: Andrew Purtell <ap...@apache.org>
Authored: Fri May 23 13:49:57 2014 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Fri May 23 13:50:12 2014 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java   | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/30aab8b5/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
index 20cf767..d8ee65e 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
@@ -56,6 +56,8 @@ import com.google.common.annotations.VisibleForTesting;
  * <tr><td>NaN</td><td>0x25</td></tr>
  * <tr><td>fixed-length 32-bit integer</td><td>0x27, I</td></tr>
  * <tr><td>fixed-length 64-bit integer</td><td>0x28, I</td></tr>
+ * <tr><td>fixed-length 8-bit integer</td><td>0x29</td></tr>
+ * <tr><td>fixed-length 16-bit integer</td><td>0x2a</td></tr>
  * <tr><td>fixed-length 32-bit float</td><td>0x30, F</td></tr>
  * <tr><td>fixed-length 64-bit float</td><td>0x31, F</td></tr>
  * <tr><td>TEXT</td><td>0x33, T</td></tr>


[4/4] git commit: HBASE-11161 Provide example of POJO encoding with protobuf (Nick Dimiduk)

Posted by ap...@apache.org.
HBASE-11161 Provide example of POJO encoding with protobuf (Nick Dimiduk)


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

Branch: refs/heads/0.98
Commit: d73711c800415250e5543cf22db486074c8ce815
Parents: 2aed26a
Author: Andrew Purtell <ap...@apache.org>
Authored: Fri May 23 13:53:23 2014 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Fri May 23 13:54:15 2014 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/types/PBType.java   | 94 ++++++++++++++++++++
 .../org/apache/hadoop/hbase/types/PBCell.java   | 77 ++++++++++++++++
 .../apache/hadoop/hbase/types/TestPBCell.java   | 53 +++++++++++
 3 files changed, 224 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d73711c8/hbase-common/src/main/java/org/apache/hadoop/hbase/types/PBType.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/PBType.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/PBType.java
new file mode 100644
index 0000000..91e9524
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/PBType.java
@@ -0,0 +1,94 @@
+/**
+ * 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.types;
+
+import com.google.protobuf.CodedInputStream;
+import com.google.protobuf.CodedOutputStream;
+import com.google.protobuf.Message;
+import org.apache.hadoop.hbase.util.Order;
+import org.apache.hadoop.hbase.util.PositionedByteRange;
+
+/**
+ * A base-class for {@link DataType} implementations backed by protobuf. See
+ * {@code PBKeyValue} in {@code hbase-examples} module.
+ */
+public abstract class PBType<T extends Message> implements DataType<T> {
+  @Override
+  public boolean isOrderPreserving() {
+    return false;
+  }
+
+  @Override
+  public Order getOrder() {
+    return null;
+  }
+
+  @Override
+  public boolean isNullable() {
+    return false;
+  }
+
+  @Override
+  public boolean isSkippable() {
+    return true;
+  }
+
+  @Override
+  public int encodedLength(T val) {
+    return val.getSerializedSize();
+  }
+
+  /**
+   * Create a {@link CodedInputStream} from a {@link PositionedByteRange}. Be sure to update
+   * {@code src}'s position after consuming from the stream.
+   * <p>For example:
+   * <pre>
+   * Foo.Builder builder = ...
+   * CodedInputStream is = inputStreamFromByteRange(src);
+   * Foo ret = builder.mergeFrom(is).build();
+   * src.setPosition(src.getPosition() + is.getTotalBytesRead());
+   * </pre>
+   */
+  public static CodedInputStream inputStreamFromByteRange(PositionedByteRange src) {
+    return CodedInputStream.newInstance(
+      src.getBytes(),
+      src.getOffset() + src.getPosition(),
+      src.getRemaining());
+  }
+
+  /**
+   * Create a {@link CodedOutputStream} from a {@link PositionedByteRange}. Be sure to update
+   * {@code dst}'s position after writing to the stream.
+   * <p>For example:
+   * <pre>
+   * CodedOutputStream os = outputStreamFromByteRange(dst);
+   * int before = os.spaceLeft(), after, written;
+   * val.writeTo(os);
+   * after = os.spaceLeft();
+   * written = before - after;
+   * dst.setPosition(dst.getPosition() + written);
+   * </pre>
+   */
+  public static CodedOutputStream outputStreamFromByteRange(PositionedByteRange dst) {
+    return CodedOutputStream.newInstance(
+      dst.getBytes(),
+      dst.getOffset() + dst.getPosition(),
+      dst.getRemaining()
+    );
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d73711c8/hbase-examples/src/main/java/org/apache/hadoop/hbase/types/PBCell.java
----------------------------------------------------------------------
diff --git a/hbase-examples/src/main/java/org/apache/hadoop/hbase/types/PBCell.java b/hbase-examples/src/main/java/org/apache/hadoop/hbase/types/PBCell.java
new file mode 100644
index 0000000..96ecf28
--- /dev/null
+++ b/hbase-examples/src/main/java/org/apache/hadoop/hbase/types/PBCell.java
@@ -0,0 +1,77 @@
+/**
+ * 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.types;
+
+import com.google.protobuf.CodedInputStream;
+import com.google.protobuf.CodedOutputStream;
+import org.apache.hadoop.hbase.protobuf.generated.CellProtos;
+import org.apache.hadoop.hbase.util.PositionedByteRange;
+
+import java.io.IOException;
+
+/**
+ * An example for using protobuf objects with {@link DataType} API.
+ */
+public class PBCell extends PBType<CellProtos.Cell> {
+  @Override
+  public Class<CellProtos.Cell> encodedClass() {
+    return CellProtos.Cell.class;
+  }
+
+  @Override
+  public int skip(PositionedByteRange src) {
+    CellProtos.Cell.Builder builder = CellProtos.Cell.newBuilder();
+    CodedInputStream is = inputStreamFromByteRange(src);
+    try {
+      builder.mergeFrom(is);
+      int consumed = is.getTotalBytesRead();
+      src.setPosition(src.getPosition() + consumed);
+      return consumed;
+    } catch (IOException e) {
+      throw new RuntimeException("Error while skipping type.", e);
+    }
+  }
+
+  @Override
+  public CellProtos.Cell decode(PositionedByteRange src) {
+    CellProtos.Cell.Builder builder = CellProtos.Cell.newBuilder();
+    CodedInputStream is = inputStreamFromByteRange(src);
+    try {
+      CellProtos.Cell ret = builder.mergeFrom(is).build();
+      src.setPosition(src.getPosition() + is.getTotalBytesRead());
+      return ret;
+    } catch (IOException e) {
+      throw new RuntimeException("Error while decoding type.", e);
+    }
+  }
+
+  @Override
+  public int encode(PositionedByteRange dst, CellProtos.Cell val) {
+    CodedOutputStream os = outputStreamFromByteRange(dst);
+    try {
+      int before = os.spaceLeft(), after, written;
+      val.writeTo(os);
+      after = os.spaceLeft();
+      written = before - after;
+      dst.setPosition(dst.getPosition() + written);
+      return written;
+    } catch (IOException e) {
+      throw new RuntimeException("Error while encoding type.", e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/d73711c8/hbase-examples/src/test/java/org/apache/hadoop/hbase/types/TestPBCell.java
----------------------------------------------------------------------
diff --git a/hbase-examples/src/test/java/org/apache/hadoop/hbase/types/TestPBCell.java b/hbase-examples/src/test/java/org/apache/hadoop/hbase/types/TestPBCell.java
new file mode 100644
index 0000000..952a319
--- /dev/null
+++ b/hbase-examples/src/test/java/org/apache/hadoop/hbase/types/TestPBCell.java
@@ -0,0 +1,53 @@
+/**
+ * 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.types;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.CellProtos;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.PositionedByteRange;
+import org.apache.hadoop.hbase.util.SimplePositionedByteRange;
+import org.junit.Test;
+
+public class TestPBCell {
+
+  private static final PBCell CODEC = new PBCell();
+
+  /**
+   * Basic test to verify utility methods in {@link PBType} and delegation to protobuf works.
+   */
+  @Test
+  public void testRoundTrip() {
+    final Cell cell = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"),
+      Bytes.toBytes("qual"), Bytes.toBytes("val"));
+    CellProtos.Cell c = ProtobufUtil.toCell(cell), decoded;
+    PositionedByteRange pbr = new SimplePositionedByteRange(c.getSerializedSize());
+    pbr.setPosition(0);
+    int encodedLength = CODEC.encode(pbr, c);
+    pbr.setPosition(0);
+    decoded = CODEC.decode(pbr);
+    assertEquals(encodedLength, pbr.getPosition());
+    assertTrue(CellComparator.equals(cell, ProtobufUtil.toCell(decoded)));
+  }
+}


[2/4] git commit: HBASE-11227 Mention 8- and 16-bit fixed-with encodings in OrderedBytes docstring (Nick Dimiduk)

Posted by ap...@apache.org.
HBASE-11227 Mention 8- and 16-bit fixed-with encodings in OrderedBytes docstring (Nick Dimiduk)


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

Branch: refs/heads/0.98
Commit: 2aed26ae6f9f3b7e961d0d92aebad41ecd2b200c
Parents: b985e4f
Author: Andrew Purtell <ap...@apache.org>
Authored: Fri May 23 13:49:57 2014 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Fri May 23 13:50:34 2014 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java   | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/2aed26ae/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
index 20cf767..d8ee65e 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/OrderedBytes.java
@@ -56,6 +56,8 @@ import com.google.common.annotations.VisibleForTesting;
  * <tr><td>NaN</td><td>0x25</td></tr>
  * <tr><td>fixed-length 32-bit integer</td><td>0x27, I</td></tr>
  * <tr><td>fixed-length 64-bit integer</td><td>0x28, I</td></tr>
+ * <tr><td>fixed-length 8-bit integer</td><td>0x29</td></tr>
+ * <tr><td>fixed-length 16-bit integer</td><td>0x2a</td></tr>
  * <tr><td>fixed-length 32-bit float</td><td>0x30, F</td></tr>
  * <tr><td>fixed-length 64-bit float</td><td>0x31, F</td></tr>
  * <tr><td>TEXT</td><td>0x33, T</td></tr>


[3/4] git commit: HBASE-11161 Provide example of POJO encoding with protobuf (Nick Dimiduk)

Posted by ap...@apache.org.
HBASE-11161 Provide example of POJO encoding with protobuf (Nick Dimiduk)


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

Branch: refs/heads/master
Commit: 8b145419edac5d5d29e0e7d5718f60635e355500
Parents: 30aab8b
Author: Andrew Purtell <ap...@apache.org>
Authored: Fri May 23 13:53:23 2014 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Fri May 23 13:53:23 2014 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/types/PBType.java   | 94 ++++++++++++++++++++
 .../org/apache/hadoop/hbase/types/PBCell.java   | 77 ++++++++++++++++
 .../apache/hadoop/hbase/types/TestPBCell.java   | 53 +++++++++++
 3 files changed, 224 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/8b145419/hbase-common/src/main/java/org/apache/hadoop/hbase/types/PBType.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/types/PBType.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/PBType.java
new file mode 100644
index 0000000..91e9524
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/types/PBType.java
@@ -0,0 +1,94 @@
+/**
+ * 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.types;
+
+import com.google.protobuf.CodedInputStream;
+import com.google.protobuf.CodedOutputStream;
+import com.google.protobuf.Message;
+import org.apache.hadoop.hbase.util.Order;
+import org.apache.hadoop.hbase.util.PositionedByteRange;
+
+/**
+ * A base-class for {@link DataType} implementations backed by protobuf. See
+ * {@code PBKeyValue} in {@code hbase-examples} module.
+ */
+public abstract class PBType<T extends Message> implements DataType<T> {
+  @Override
+  public boolean isOrderPreserving() {
+    return false;
+  }
+
+  @Override
+  public Order getOrder() {
+    return null;
+  }
+
+  @Override
+  public boolean isNullable() {
+    return false;
+  }
+
+  @Override
+  public boolean isSkippable() {
+    return true;
+  }
+
+  @Override
+  public int encodedLength(T val) {
+    return val.getSerializedSize();
+  }
+
+  /**
+   * Create a {@link CodedInputStream} from a {@link PositionedByteRange}. Be sure to update
+   * {@code src}'s position after consuming from the stream.
+   * <p>For example:
+   * <pre>
+   * Foo.Builder builder = ...
+   * CodedInputStream is = inputStreamFromByteRange(src);
+   * Foo ret = builder.mergeFrom(is).build();
+   * src.setPosition(src.getPosition() + is.getTotalBytesRead());
+   * </pre>
+   */
+  public static CodedInputStream inputStreamFromByteRange(PositionedByteRange src) {
+    return CodedInputStream.newInstance(
+      src.getBytes(),
+      src.getOffset() + src.getPosition(),
+      src.getRemaining());
+  }
+
+  /**
+   * Create a {@link CodedOutputStream} from a {@link PositionedByteRange}. Be sure to update
+   * {@code dst}'s position after writing to the stream.
+   * <p>For example:
+   * <pre>
+   * CodedOutputStream os = outputStreamFromByteRange(dst);
+   * int before = os.spaceLeft(), after, written;
+   * val.writeTo(os);
+   * after = os.spaceLeft();
+   * written = before - after;
+   * dst.setPosition(dst.getPosition() + written);
+   * </pre>
+   */
+  public static CodedOutputStream outputStreamFromByteRange(PositionedByteRange dst) {
+    return CodedOutputStream.newInstance(
+      dst.getBytes(),
+      dst.getOffset() + dst.getPosition(),
+      dst.getRemaining()
+    );
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8b145419/hbase-examples/src/main/java/org/apache/hadoop/hbase/types/PBCell.java
----------------------------------------------------------------------
diff --git a/hbase-examples/src/main/java/org/apache/hadoop/hbase/types/PBCell.java b/hbase-examples/src/main/java/org/apache/hadoop/hbase/types/PBCell.java
new file mode 100644
index 0000000..96ecf28
--- /dev/null
+++ b/hbase-examples/src/main/java/org/apache/hadoop/hbase/types/PBCell.java
@@ -0,0 +1,77 @@
+/**
+ * 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.types;
+
+import com.google.protobuf.CodedInputStream;
+import com.google.protobuf.CodedOutputStream;
+import org.apache.hadoop.hbase.protobuf.generated.CellProtos;
+import org.apache.hadoop.hbase.util.PositionedByteRange;
+
+import java.io.IOException;
+
+/**
+ * An example for using protobuf objects with {@link DataType} API.
+ */
+public class PBCell extends PBType<CellProtos.Cell> {
+  @Override
+  public Class<CellProtos.Cell> encodedClass() {
+    return CellProtos.Cell.class;
+  }
+
+  @Override
+  public int skip(PositionedByteRange src) {
+    CellProtos.Cell.Builder builder = CellProtos.Cell.newBuilder();
+    CodedInputStream is = inputStreamFromByteRange(src);
+    try {
+      builder.mergeFrom(is);
+      int consumed = is.getTotalBytesRead();
+      src.setPosition(src.getPosition() + consumed);
+      return consumed;
+    } catch (IOException e) {
+      throw new RuntimeException("Error while skipping type.", e);
+    }
+  }
+
+  @Override
+  public CellProtos.Cell decode(PositionedByteRange src) {
+    CellProtos.Cell.Builder builder = CellProtos.Cell.newBuilder();
+    CodedInputStream is = inputStreamFromByteRange(src);
+    try {
+      CellProtos.Cell ret = builder.mergeFrom(is).build();
+      src.setPosition(src.getPosition() + is.getTotalBytesRead());
+      return ret;
+    } catch (IOException e) {
+      throw new RuntimeException("Error while decoding type.", e);
+    }
+  }
+
+  @Override
+  public int encode(PositionedByteRange dst, CellProtos.Cell val) {
+    CodedOutputStream os = outputStreamFromByteRange(dst);
+    try {
+      int before = os.spaceLeft(), after, written;
+      val.writeTo(os);
+      after = os.spaceLeft();
+      written = before - after;
+      dst.setPosition(dst.getPosition() + written);
+      return written;
+    } catch (IOException e) {
+      throw new RuntimeException("Error while encoding type.", e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/8b145419/hbase-examples/src/test/java/org/apache/hadoop/hbase/types/TestPBCell.java
----------------------------------------------------------------------
diff --git a/hbase-examples/src/test/java/org/apache/hadoop/hbase/types/TestPBCell.java b/hbase-examples/src/test/java/org/apache/hadoop/hbase/types/TestPBCell.java
new file mode 100644
index 0000000..952a319
--- /dev/null
+++ b/hbase-examples/src/test/java/org/apache/hadoop/hbase/types/TestPBCell.java
@@ -0,0 +1,53 @@
+/**
+ * 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.types;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.CellProtos;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.PositionedByteRange;
+import org.apache.hadoop.hbase.util.SimplePositionedByteRange;
+import org.junit.Test;
+
+public class TestPBCell {
+
+  private static final PBCell CODEC = new PBCell();
+
+  /**
+   * Basic test to verify utility methods in {@link PBType} and delegation to protobuf works.
+   */
+  @Test
+  public void testRoundTrip() {
+    final Cell cell = new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("fam"),
+      Bytes.toBytes("qual"), Bytes.toBytes("val"));
+    CellProtos.Cell c = ProtobufUtil.toCell(cell), decoded;
+    PositionedByteRange pbr = new SimplePositionedByteRange(c.getSerializedSize());
+    pbr.setPosition(0);
+    int encodedLength = CODEC.encode(pbr, c);
+    pbr.setPosition(0);
+    decoded = CODEC.decode(pbr);
+    assertEquals(encodedLength, pbr.getPosition());
+    assertTrue(CellComparator.equals(cell, ProtobufUtil.toCell(decoded)));
+  }
+}