You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ra...@apache.org on 2017/11/24 06:52:43 UTC

[2/2] hbase git commit: HBASE-19092 Make Tag IA.LimitedPrivate and expose for CPs (Ram)

HBASE-19092 Make Tag IA.LimitedPrivate and expose for CPs (Ram)


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

Branch: refs/heads/master
Commit: 73e3af00e94f0be12dd6e399d5e72966311ae3fe
Parents: 8572364
Author: Vasudevan <ra...@intel.com>
Authored: Fri Nov 24 12:21:38 2017 +0530
Committer: Vasudevan <ra...@intel.com>
Committed: Fri Nov 24 12:21:38 2017 +0530

----------------------------------------------------------------------
 .../apache/hadoop/hbase/client/Mutation.java    |  13 +-
 .../hadoop/hbase/protobuf/ProtobufUtil.java     |   6 +-
 .../hbase/shaded/protobuf/ProtobufUtil.java     |   3 +-
 .../main/java/org/apache/hadoop/hbase/Cell.java |   5 +-
 .../java/org/apache/hadoop/hbase/CellUtil.java  |  44 +++-
 .../org/apache/hadoop/hbase/ExtendedCell.java   |   6 +-
 .../hadoop/hbase/ExtendedCellBuilder.java       |   9 +-
 .../hbase/ExtendedCellBuilderFactory.java       |  15 +-
 .../hadoop/hbase/ExtendedCellBuilderImpl.java   |  13 +-
 .../hadoop/hbase/IndividualBytesFieldCell.java  |   2 +-
 .../hbase/IndividualBytesFieldCellBuilder.java  |   9 +
 .../java/org/apache/hadoop/hbase/KeyValue.java  |  17 +-
 .../apache/hadoop/hbase/KeyValueBuilder.java    |  11 +-
 .../apache/hadoop/hbase/PrivateCellUtil.java    | 231 ++++++++-----------
 .../java/org/apache/hadoop/hbase/RawCell.java   |  68 ++++++
 .../main/java/org/apache/hadoop/hbase/Tag.java  | 136 ++++++++++-
 .../java/org/apache/hadoop/hbase/TagUtil.java   | 176 +-------------
 .../hadoop/hbase/TestByteBufferKeyValue.java    |  23 +-
 .../org/apache/hadoop/hbase/TestKeyValue.java   |  12 +-
 .../org/apache/hadoop/hbase/TestTagUtil.java    |   4 +-
 .../hbase/codec/TestCellCodecWithTags.java      |  28 +--
 .../hbase/codec/TestKeyValueCodecWithTags.java  |  28 +--
 .../hbase/mapreduce/TestHFileOutputFormat2.java |   5 +-
 .../RegionCoprocessorEnvironment.java           |   8 +
 .../hbase/io/hfile/HFilePrettyPrinter.java      |  15 +-
 .../org/apache/hadoop/hbase/mob/MobUtils.java   |  16 +-
 .../compactions/PartitionedMobCompactor.java    |   6 +-
 .../hadoop/hbase/regionserver/HMobStore.java    |  14 +-
 .../hadoop/hbase/regionserver/HRegion.java      |   2 +-
 .../regionserver/RegionCoprocessorHost.java     |  13 +-
 .../querymatcher/ScanQueryMatcher.java          |   7 +-
 .../security/access/AccessControlLists.java     |  17 +-
 .../hbase/security/access/AccessController.java |  17 +-
 .../DefaultVisibilityLabelServiceImpl.java      |   2 +-
 .../visibility/VisibilityController.java        |  20 +-
 .../security/visibility/VisibilityUtils.java    |   6 +-
 .../hadoop/hbase/wal/WALPrettyPrinter.java      |  11 +-
 .../apache/hadoop/hbase/io/hfile/TagUsage.java  |   4 +
 .../hadoop/hbase/io/hfile/TestSeekTo.java       |  22 +-
 .../hadoop/hbase/protobuf/TestProtobufUtil.java |   8 +-
 .../TestStoreFileScannerWithTagCompression.java |   3 +-
 .../hadoop/hbase/regionserver/TestTags.java     |  21 +-
 .../wal/TestWALCellCodecWithCompression.java    |  11 +-
 .../replication/TestReplicationWithTags.java    |   8 +-
 .../security/token/TestTokenAuthentication.java |   6 +
 .../ExpAsStringVisibilityLabelServiceImpl.java  |   8 +-
 ...sibilityLabelReplicationWithExpAsString.java |   3 +-
 .../TestVisibilityLabelsReplication.java        |  17 +-
 .../apache/hadoop/hbase/util/HFileTestUtil.java |  16 +-
 .../hadoop/hbase/thrift2/ThriftUtilities.java   |  11 +-
 50 files changed, 618 insertions(+), 538 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/73e3af00/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
index dd2c2f0..4ba499f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Mutation.java
@@ -34,11 +34,9 @@ import org.apache.hadoop.hbase.CellScannable;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.PrivateCellUtil;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.PrivateCellUtil;
 import org.apache.hadoop.hbase.Tag;
-import org.apache.hadoop.hbase.TagUtil;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.io.HeapSize;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@@ -48,14 +46,14 @@ import org.apache.hadoop.hbase.security.access.AccessControlUtil;
 import org.apache.hadoop.hbase.security.access.Permission;
 import org.apache.hadoop.hbase.security.visibility.CellVisibility;
 import org.apache.hadoop.hbase.security.visibility.VisibilityConstants;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.ClassSize;
-
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.ArrayListMultimap;
 import org.apache.hadoop.hbase.shaded.com.google.common.collect.ListMultimap;
 import org.apache.hadoop.hbase.shaded.com.google.common.io.ByteArrayDataInput;
 import org.apache.hadoop.hbase.shaded.com.google.common.io.ByteArrayDataOutput;
 import org.apache.hadoop.hbase.shaded.com.google.common.io.ByteStreams;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.ClassSize;
+import org.apache.yetus.audience.InterfaceAudience;
 
 @InterfaceAudience.Public
 public abstract class Mutation extends OperationWithAttributes implements Row, CellScannable,
@@ -230,7 +228,8 @@ public abstract class Mutation extends OperationWithAttributes implements Row, C
     if (tags != null) {
       List<String> tagsString = new ArrayList<>(tags.size());
       for (Tag t : tags) {
-        tagsString.add((t.getType()) + ":" + Bytes.toStringBinary(TagUtil.cloneValue(t)));
+        tagsString
+            .add((t.getType()) + ":" + Bytes.toStringBinary(Tag.cloneValue(t)));
       }
       stringMap.put("tag", tagsString);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/73e3af00/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
index d86fc62..bc49cb0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
@@ -30,7 +30,6 @@ import java.util.NavigableSet;
 import java.util.function.Function;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.CacheEvictionStats;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellBuilderType;
 import org.apache.hadoop.hbase.CellScanner;
@@ -46,7 +45,6 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.TagUtil;
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Append;
 import org.apache.hadoop.hbase.client.Consistency;
 import org.apache.hadoop.hbase.client.Delete;
@@ -91,6 +89,7 @@ import org.apache.hadoop.hbase.util.DynamicClassLoader;
 import org.apache.hadoop.hbase.util.ExceptionUtil;
 import org.apache.hadoop.hbase.util.Methods;
 import org.apache.hadoop.ipc.RemoteException;
+import org.apache.yetus.audience.InterfaceAudience;
 
 import com.google.protobuf.ByteString;
 import com.google.protobuf.CodedInputStream;
@@ -538,7 +537,8 @@ public final class ProtobufUtil {
                       .setTags(allTagsBytes)
                       .build());
             } else {
-              List<Tag> tags = TagUtil.asList(allTagsBytes, 0, (short)allTagsBytes.length);
+              List<Tag> tags =
+                  TagUtil.asList(allTagsBytes, 0, (short) allTagsBytes.length);
               Tag[] tagsArray = new Tag[tags.size()];
               put.addImmutable(family, qualifier, ts, value, tags.toArray(tagsArray));
             }

http://git-wip-us.apache.org/repos/asf/hbase/blob/73e3af00/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
index 3dd1d47..5971b3c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
@@ -673,7 +673,8 @@ public final class ProtobufUtil {
                       .setTags(allTagsBytes)
                       .build());
             } else {
-              List<Tag> tags = TagUtil.asList(allTagsBytes, 0, (short)allTagsBytes.length);
+              List<Tag> tags =
+                  TagUtil.asList(allTagsBytes, 0, (short) allTagsBytes.length);
               Tag[] tagsArray = new Tag[tags.size()];
               put.addImmutable(family, qualifier, ts, value, tags.toArray(tagsArray));
             }

http://git-wip-us.apache.org/repos/asf/hbase/blob/73e3af00/hbase-common/src/main/java/org/apache/hadoop/hbase/Cell.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/Cell.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/Cell.java
index f5833c8..2b99823 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/Cell.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/Cell.java
@@ -184,9 +184,10 @@ public interface Cell {
   /**
    * HBase internally uses 2 bytes to store tags length in Cell.
    * As the tags length is always a non-negative number, to make good use of the sign bit,
-   * the max of tags length is defined as {@link TagUtil#MAX_TAGS_LENGTH}, which is 2 * Short.MAX_VALUE + 1 = 65535.
+   * the max of tags length is defined 2 * Short.MAX_VALUE + 1 = 65535.
    * As a result, the return type is int, because a short is not capable of handling that.
-   * Please note that even if the return type is int, the max tags length is far less than Integer.MAX_VALUE.
+   * Please note that even if the return type is int, the max tags length is far
+   * less than Integer.MAX_VALUE.
    *
    * @return the total length of the tags in the Cell.
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/73e3af00/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
index 52eb8fa..9fabfdd 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellUtil.java
@@ -18,10 +18,10 @@
 
 package org.apache.hadoop.hbase;
 
-import static org.apache.hadoop.hbase.Tag.TAG_LENGTH_SIZE;
 import static org.apache.hadoop.hbase.KeyValue.COLUMN_FAMILY_DELIMITER;
-import static org.apache.hadoop.hbase.KeyValue.getDelimiter;
 import static org.apache.hadoop.hbase.KeyValue.COLUMN_FAMILY_DELIM_ARRAY;
+import static org.apache.hadoop.hbase.KeyValue.getDelimiter;
+import static org.apache.hadoop.hbase.Tag.TAG_LENGTH_SIZE;
 
 import java.io.DataOutput;
 import java.io.DataOutputStream;
@@ -33,17 +33,16 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map.Entry;
 import java.util.NavigableMap;
+import java.util.Optional;
 
 import org.apache.hadoop.hbase.KeyValue.Type;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceAudience.Private;
-
-import com.google.common.annotations.VisibleForTesting;
-
 import org.apache.hadoop.hbase.io.HeapSize;
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.hbase.util.ByteBufferUtils;
 import org.apache.hadoop.hbase.util.ByteRange;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceAudience.Private;
 
 /**
  * Utility methods helpful for slinging {@link Cell} instances. Some methods below are for internal
@@ -129,6 +128,7 @@ public final class CellUtil {
 
   /**
    * @deprecated As of HBase-2.0. Will be removed in HBase-3.0.
+   *             Use {@link RawCell#cloneTags()}
    */
   @Deprecated
   public static byte[] cloneTags(Cell cell) {
@@ -145,7 +145,7 @@ public final class CellUtil {
    */
   @Deprecated
   public static byte[] getTagArray(Cell cell) {
-    return PrivateCellUtil.getTagsArray(cell);
+    return PrivateCellUtil.cloneTags(cell);
   }
 
   /**
@@ -560,15 +560,24 @@ public final class CellUtil {
   }
 
   /**
+   * Note : Now only CPs can create cell with tags using the CP environment
    * @return A new cell which is having the extra tags also added to it.
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
+   *             Use CP environment to build Cell using {@link ExtendedCellBuilder}
+   *        
    */
+  @Deprecated
   public static Cell createCell(Cell cell, List<Tag> tags) {
-    return createCell(cell, TagUtil.fromList(tags));
+    return createCell(cell, Tag.fromList(tags));
   }
 
   /**
+   * Now only CPs can create cell with tags using the CP environment
    * @return A new cell which is having the extra tags also added to it.
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
+   *            Use CP environment to build Cell using {@link ExtendedCellBuilder}
    */
+  @Deprecated
   public static Cell createCell(Cell cell, byte[] tags) {
     if (cell instanceof ByteBufferCell) {
       return new PrivateCellUtil.TagRewriteByteBufferCell((ByteBufferCell) cell, tags);
@@ -576,6 +585,12 @@ public final class CellUtil {
     return new PrivateCellUtil.TagRewriteCell(cell, tags);
   }
 
+  /**
+   * Now only CPs can create cell with tags using the CP environment
+   * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
+   *             Use CP environment to build Cell using {@link ExtendedCellBuilder}
+   */
+  @Deprecated
   public static Cell createCell(Cell cell, byte[] value, byte[] tags) {
     if (cell instanceof ByteBufferCell) {
       return new PrivateCellUtil.ValueAndTagRewriteByteBufferCell((ByteBufferCell) cell, value,
@@ -1054,6 +1069,7 @@ public final class CellUtil {
    * @param cell The Cell
    * @return Tags in the given Cell as a List
    * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
+   *             Use {@link RawCell#getTags()}
    */
   @Deprecated
   public static List<Tag> getTags(Cell cell) {
@@ -1067,10 +1083,16 @@ public final class CellUtil {
    * @param type Type of the Tag to retrieve
    * @return null if there is no tag of the passed in tag type
    * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
+   *             Use {@link RawCell#getTag(byte)}
    */
   @Deprecated
-  public static Tag getTag(Cell cell, byte type){
-    return PrivateCellUtil.getTag(cell, type);
+  public static Tag getTag(Cell cell, byte type) {
+    Optional<Tag> tag = PrivateCellUtil.getTag(cell, type);
+    if (tag.isPresent()) {
+      return tag.get();
+    } else {
+      return null;
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/73e3af00/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCell.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCell.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCell.java
index 4d16fca..b5ce095 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCell.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCell.java
@@ -21,8 +21,8 @@ import java.io.IOException;
 import java.io.OutputStream;
 import java.nio.ByteBuffer;
 
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.io.HeapSize;
+import org.apache.yetus.audience.InterfaceAudience;
 
 /**
  * Extension to {@link Cell} with server side required functions. Server side Cell implementations
@@ -30,8 +30,8 @@ import org.apache.hadoop.hbase.io.HeapSize;
  * @see SettableSequenceId
  * @see SettableTimestamp
  */
-@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
-public interface ExtendedCell extends Cell, SettableSequenceId, SettableTimestamp, HeapSize,
+@InterfaceAudience.Private
+public interface ExtendedCell extends RawCell, SettableSequenceId, SettableTimestamp, HeapSize,
     Cloneable {
 
   public static int CELL_NOT_BASED_ON_CHUNK = -1;

http://git-wip-us.apache.org/repos/asf/hbase/blob/73e3af00/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCellBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCellBuilder.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCellBuilder.java
index 41d204c..57fa44e 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCellBuilder.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCellBuilder.java
@@ -26,7 +26,7 @@ import org.apache.yetus.audience.InterfaceAudience;
  * Use {@link ExtendedCellBuilderFactory} to get ExtendedCellBuilder instance.
  * TODO: ditto for ByteBufferCell?
  */
-@InterfaceAudience.Private
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
 public interface ExtendedCellBuilder extends CellBuilder {
   @Override
   ExtendedCellBuilder setRow(final byte[] row);
@@ -62,8 +62,15 @@ public interface ExtendedCellBuilder extends CellBuilder {
   @Override
   ExtendedCellBuilder clear();
 
+  // TODO : While creating RawCellBuilder allow 'Tag' to be passed instead of byte[]
   ExtendedCellBuilder setTags(final byte[] tags);
+  // TODO : While creating RawCellBuilder allow 'Tag' to be passed instead of byte[]
   ExtendedCellBuilder setTags(final byte[] tags, int tagsOffset, int tagsLength);
 
+  /**
+   * Internal usage. Be careful before you use this while building a cell
+   * @param seqId set the seqId
+   * @return the current ExternalCellBuilder
+   */
   ExtendedCellBuilder setSequenceId(final long seqId);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/73e3af00/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCellBuilderFactory.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCellBuilderFactory.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCellBuilderFactory.java
index 87f84c9..38778fb 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCellBuilderFactory.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCellBuilderFactory.java
@@ -25,11 +25,24 @@ import org.apache.yetus.audience.InterfaceAudience;
 public final class ExtendedCellBuilderFactory {
 
   public static ExtendedCellBuilder create(CellBuilderType type) {
+    return create(type, true);
+  }
+
+  /**
+   * Allows creating a cell with the given CellBuilderType.
+   * @param type the type of CellBuilder(DEEP_COPY or SHALLOW_COPY).
+   * @param allowSeqIdUpdate if seqId can be updated. CPs are not allowed to update
+   *        the seqId
+   * @return the cell that is created
+   */
+  public static ExtendedCellBuilder create(CellBuilderType type, boolean allowSeqIdUpdate) {
     switch (type) {
       case SHALLOW_COPY:
+        // CPs are not allowed to update seqID and they always use DEEP_COPY. So we have not
+        // passing 'allowSeqIdUpdate' to IndividualBytesFieldCellBuilder
         return new IndividualBytesFieldCellBuilder();
       case DEEP_COPY:
-        return new KeyValueBuilder();
+        return new KeyValueBuilder(allowSeqIdUpdate);
       default:
         throw new UnsupportedOperationException("The type:" + type + " is unsupported");
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/73e3af00/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCellBuilderImpl.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCellBuilderImpl.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCellBuilderImpl.java
index c595e2c..4dfb399 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCellBuilderImpl.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ExtendedCellBuilderImpl.java
@@ -40,6 +40,12 @@ public abstract class ExtendedCellBuilderImpl implements ExtendedCellBuilder {
   protected byte[] tags = null;
   protected int tagsOffset = 0;
   protected int tagsLength = 0;
+  // Will go away once we do with RawCellBuilder
+  protected boolean allowSeqIdUpdate = false;
+
+  public ExtendedCellBuilderImpl(boolean allowSeqIdUpdate) {
+    this.allowSeqIdUpdate = allowSeqIdUpdate;
+  }
 
   @Override
   public ExtendedCellBuilder setRow(final byte[] row) {
@@ -126,8 +132,11 @@ public abstract class ExtendedCellBuilderImpl implements ExtendedCellBuilder {
 
   @Override
   public ExtendedCellBuilder setSequenceId(final long seqId) {
-    this.seqId = seqId;
-    return this;
+    if (allowSeqIdUpdate) {
+      this.seqId = seqId;
+      return this;
+    }
+    throw new UnsupportedOperationException("SeqId cannot be set on this cell");
   }
 
   private void checkBeforeBuild() {

http://git-wip-us.apache.org/repos/asf/hbase/blob/73e3af00/hbase-common/src/main/java/org/apache/hadoop/hbase/IndividualBytesFieldCell.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/IndividualBytesFieldCell.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/IndividualBytesFieldCell.java
index a15843c..6233553 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/IndividualBytesFieldCell.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/IndividualBytesFieldCell.java
@@ -94,7 +94,7 @@ public class IndividualBytesFieldCell implements ExtendedCell {
     }
 
     // Check tags
-    TagUtil.checkForTagsLength(tagsLength);
+    RawCell.checkForTagsLength(tagsLength);
     checkArrayBounds(row, rOffset, rLength);
     checkArrayBounds(family, fOffset, fLength);
     checkArrayBounds(qualifier, qOffset, qLength);

http://git-wip-us.apache.org/repos/asf/hbase/blob/73e3af00/hbase-common/src/main/java/org/apache/hadoop/hbase/IndividualBytesFieldCellBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/IndividualBytesFieldCellBuilder.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/IndividualBytesFieldCellBuilder.java
index 2e160f1..62febf8 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/IndividualBytesFieldCellBuilder.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/IndividualBytesFieldCellBuilder.java
@@ -21,6 +21,15 @@ import org.apache.yetus.audience.InterfaceAudience;
 
 @InterfaceAudience.Private
 class IndividualBytesFieldCellBuilder extends ExtendedCellBuilderImpl {
+
+  public IndividualBytesFieldCellBuilder() {
+    this(true);
+  }
+
+  public IndividualBytesFieldCellBuilder(boolean allowSeqIdUpdate) {
+    super(allowSeqIdUpdate);
+  }
+
   @Override
   public ExtendedCell innerBuild() {
     return new IndividualBytesFieldCell(row, rOffset, rLength,

http://git-wip-us.apache.org/repos/asf/hbase/blob/73e3af00/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
index c3a429e..fa027b4 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
@@ -34,13 +34,12 @@ import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.hbase.util.ByteBufferUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.ClassSize;
 import org.apache.hadoop.io.RawComparator;
-
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
+import org.apache.yetus.audience.InterfaceAudience;
 /**
  * An HBase Key/Value. This is the fundamental HBase Type.
  * <p>
@@ -766,7 +765,7 @@ public class KeyValue implements ExtendedCell {
     if (qlength > Integer.MAX_VALUE - rlength - flength) {
       throw new IllegalArgumentException("Qualifier > " + Integer.MAX_VALUE);
     }
-    TagUtil.checkForTagsLength(tagsLength);
+    RawCell.checkForTagsLength(tagsLength);
     // Key length
     long longkeylength = getKeyDataStructureSize(rlength, flength, qlength);
     if (longkeylength > Integer.MAX_VALUE) {
@@ -884,7 +883,7 @@ public class KeyValue implements ExtendedCell {
         tagsLength += t.getValueLength() + Tag.INFRASTRUCTURE_SIZE;
       }
     }
-    TagUtil.checkForTagsLength(tagsLength);
+    RawCell.checkForTagsLength(tagsLength);
     int keyLength = (int) getKeyDataStructureSize(rlength, flength, qlength);
     int keyValueLength = (int) getKeyValueDataStructureSize(rlength, flength, qlength, vlength,
         tagsLength);
@@ -918,7 +917,7 @@ public class KeyValue implements ExtendedCell {
         int tlen = t.getValueLength();
         pos = Bytes.putAsShort(buffer, pos, tlen + Tag.TYPE_LENGTH_SIZE);
         pos = Bytes.putByte(buffer, pos, t.getType());
-        TagUtil.copyValueTo(t, buffer, pos);
+        Tag.copyValueTo(t, buffer, pos);
         pos += tlen;
       }
     }
@@ -951,7 +950,7 @@ public class KeyValue implements ExtendedCell {
       int vlength, byte[] tags, int tagsOffset, int tagsLength) {
 
     checkParameters(row, rlength, family, flength, qlength, vlength);
-    TagUtil.checkForTagsLength(tagsLength);
+    RawCell.checkForTagsLength(tagsLength);
     // Allocate right-sized byte array.
     int keyLength = (int) getKeyDataStructureSize(rlength, flength, qlength);
     byte[] bytes = new byte[(int) getKeyValueDataStructureSize(rlength, flength, qlength, vlength,
@@ -1001,7 +1000,7 @@ public class KeyValue implements ExtendedCell {
         tagsLength += t.getValueLength() + Tag.INFRASTRUCTURE_SIZE;
       }
     }
-    TagUtil.checkForTagsLength(tagsLength);
+    RawCell.checkForTagsLength(tagsLength);
     // Allocate right-sized byte array.
     int keyLength = (int) getKeyDataStructureSize(rlength, flength, qlength);
     byte[] bytes = new byte[(int) getKeyValueDataStructureSize(rlength, flength, qlength, vlength,
@@ -1041,7 +1040,7 @@ public class KeyValue implements ExtendedCell {
         int tlen = t.getValueLength();
         pos = Bytes.putAsShort(bytes, pos, tlen + Tag.TYPE_LENGTH_SIZE);
         pos = Bytes.putByte(bytes, pos, t.getType());
-        TagUtil.copyValueTo(t, bytes, pos);
+        Tag.copyValueTo(t, bytes, pos);
         pos += tlen;
       }
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/73e3af00/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueBuilder.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueBuilder.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueBuilder.java
index 94e2522..4f01992 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueBuilder.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValueBuilder.java
@@ -21,6 +21,15 @@ import org.apache.yetus.audience.InterfaceAudience;
 
 @InterfaceAudience.Private
 class KeyValueBuilder extends ExtendedCellBuilderImpl {
+
+  KeyValueBuilder() {
+    this(true);
+  }
+
+  KeyValueBuilder(boolean allowSeqIdUpdate) {
+    super(allowSeqIdUpdate);
+  }
+
   @Override
   protected ExtendedCell innerBuild() {
     KeyValue kv = new KeyValue(row, rOffset, rLength,
@@ -28,7 +37,7 @@ class KeyValueBuilder extends ExtendedCellBuilderImpl {
             qualifier, qOffset, qLength,
             timestamp, type,
             value, vOffset, vLength,
-            tags, tagsOffset, tagsLength);
+        tags, tagsOffset, tagsLength);
     kv.setSequenceId(seqId);
     return kv;
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/73e3af00/hbase-common/src/main/java/org/apache/hadoop/hbase/PrivateCellUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/PrivateCellUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/PrivateCellUtil.java
index d70d974..1acb490 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/PrivateCellUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/PrivateCellUtil.java
@@ -29,6 +29,7 @@ import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Optional;
 
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.filter.ByteArrayComparable;
@@ -49,14 +50,12 @@ import com.google.common.annotations.VisibleForTesting;
  * rich set of APIs than those in {@link CellUtil} for internal usage.
  */
 @InterfaceAudience.Private
-// TODO : Make Tag IA.LimitedPrivate and move some of the Util methods to CP exposed Util class
-public class PrivateCellUtil {
+public final class PrivateCellUtil {
 
   /**
    * Private constructor to keep this class from being instantiated.
    */
   private PrivateCellUtil() {
-
   }
 
   /******************* ByteRange *******************************/
@@ -82,64 +81,6 @@ public class PrivateCellUtil {
     return range.set(cell.getTagsArray(), cell.getTagsOffset(), cell.getTagsLength());
   }
 
-  /**
-   * Returns tag value in a new byte array. If server-side, use {@link Tag#getValueArray()} with
-   * appropriate {@link Tag#getValueOffset()} and {@link Tag#getValueLength()} instead to save on
-   * allocations.
-   * @param cell
-   * @return tag value in a new byte array.
-   */
-  public static byte[] getTagsArray(Cell cell) {
-    byte[] output = new byte[cell.getTagsLength()];
-    copyTagsTo(cell, output, 0);
-    return output;
-  }
-
-  public static byte[] cloneTags(Cell cell) {
-    byte[] output = new byte[cell.getTagsLength()];
-    copyTagsTo(cell, output, 0);
-    return output;
-  }
-
-  /**
-   * Copies the tags info into the tag portion of the cell
-   * @param cell
-   * @param destination
-   * @param destinationOffset
-   * @return position after tags
-   */
-  public static int copyTagsTo(Cell cell, byte[] destination, int destinationOffset) {
-    int tlen = cell.getTagsLength();
-    if (cell instanceof ByteBufferCell) {
-      ByteBufferUtils.copyFromBufferToArray(destination,
-        ((ByteBufferCell) cell).getTagsByteBuffer(), ((ByteBufferCell) cell).getTagsPosition(),
-        destinationOffset, tlen);
-    } else {
-      System.arraycopy(cell.getTagsArray(), cell.getTagsOffset(), destination, destinationOffset,
-        tlen);
-    }
-    return destinationOffset + tlen;
-  }
-
-  /**
-   * Copies the tags info into the tag portion of the cell
-   * @param cell
-   * @param destination
-   * @param destinationOffset
-   * @return the position after tags
-   */
-  public static int copyTagsTo(Cell cell, ByteBuffer destination, int destinationOffset) {
-    int tlen = cell.getTagsLength();
-    if (cell instanceof ByteBufferCell) {
-      ByteBufferUtils.copyFromBufferToBuffer(((ByteBufferCell) cell).getTagsByteBuffer(),
-        destination, ((ByteBufferCell) cell).getTagsPosition(), destinationOffset, tlen);
-    } else {
-      ByteBufferUtils.copyFromArrayToBuffer(destination, destinationOffset, cell.getTagsArray(),
-        cell.getTagsOffset(), tlen);
-    }
-    return destinationOffset + tlen;
-  }
-
   /********************* misc *************************************/
 
   public static byte getRowByte(Cell cell, int index) {
@@ -168,7 +109,7 @@ public class PrivateCellUtil {
    * @return A new cell which is having the extra tags also added to it.
    */
   public static Cell createCell(Cell cell, List<Tag> tags) {
-    return createCell(cell, TagUtil.fromList(tags));
+    return createCell(cell, Tag.fromList(tags));
   }
 
   /**
@@ -653,7 +594,9 @@ public class PrivateCellUtil {
       ByteBufferUtils.putInt(out, valLen);// Value length
       int len = 2 * Bytes.SIZEOF_INT;
       len += writeFlatKey(cell, out);// Key
-      if (valLen > 0) out.write(value);// Value
+      if (valLen > 0) {
+        out.write(value);// Value
+      }
       len += valLen;
       if (withTags && tags != null) {
         // Write the tagsLength 2 bytes
@@ -787,7 +730,7 @@ public class PrivateCellUtil {
 
   /**
    * Finds if the qualifier part of the cell and the KV serialized byte[] are equal
-   * @param left
+   * @param left the cell with which we need to match the qualifier
    * @param buf the serialized keyvalue format byte[]
    * @param offset the offset of the qualifier in the byte[]
    * @param length the length of the qualifier in the byte[]
@@ -809,7 +752,9 @@ public class PrivateCellUtil {
 
   public static boolean matchingColumn(final Cell left, final byte[] fam, final int foffset,
       final int flength, final byte[] qual, final int qoffset, final int qlength) {
-    if (!matchingFamily(left, fam, foffset, flength)) return false;
+    if (!matchingFamily(left, fam, foffset, flength)) {
+      return false;
+    }
     return matchingQualifier(left, qual, qoffset, qlength);
   }
 
@@ -878,79 +823,49 @@ public class PrivateCellUtil {
     return t == Type.DeleteColumn.getCode() || t == Type.DeleteFamily.getCode();
   }
 
-  private static Iterator<Tag> tagsIterator(final ByteBuffer tags, final int offset,
-      final int length) {
-    return new Iterator<Tag>() {
-      private int pos = offset;
-      private int endOffset = offset + length - 1;
-
-      @Override
-      public boolean hasNext() {
-        return this.pos < endOffset;
-      }
-
-      @Override
-      public Tag next() {
-        if (hasNext()) {
-          int curTagLen = ByteBufferUtils.readAsInt(tags, this.pos, Tag.TAG_LENGTH_SIZE);
-          Tag tag = new ByteBufferTag(tags, pos, curTagLen + Tag.TAG_LENGTH_SIZE);
-          this.pos += Bytes.SIZEOF_SHORT + curTagLen;
-          return tag;
-        }
-        return null;
-      }
-
-      @Override
-      public void remove() {
-        throw new UnsupportedOperationException();
-      }
-    };
+  public static byte[] cloneTags(Cell cell) {
+    byte[] output = new byte[cell.getTagsLength()];
+    copyTagsTo(cell, output, 0);
+    return output;
   }
 
   /**
-   * Util method to iterate through the tags in the given cell.
-   * @param cell The Cell over which tags iterator is needed.
-   * @return iterator for the tags
+   * Copies the tags info into the tag portion of the cell
+   * @param cell
+   * @param destination
+   * @param destinationOffset
+   * @return position after tags
    */
-  public static Iterator<Tag> tagsIterator(final Cell cell) {
-    final int tagsLength = cell.getTagsLength();
-    // Save an object allocation where we can
-    if (tagsLength == 0) {
-      return TagUtil.EMPTY_TAGS_ITR;
-    }
+  public static int copyTagsTo(Cell cell, byte[] destination, int destinationOffset) {
+    int tlen = cell.getTagsLength();
     if (cell instanceof ByteBufferCell) {
-      return tagsIterator(((ByteBufferCell) cell).getTagsByteBuffer(),
-        ((ByteBufferCell) cell).getTagsPosition(), tagsLength);
+      ByteBufferUtils.copyFromBufferToArray(destination,
+        ((ByteBufferCell) cell).getTagsByteBuffer(), ((ByteBufferCell) cell).getTagsPosition(),
+        destinationOffset, tlen);
+    } else {
+      System.arraycopy(cell.getTagsArray(), cell.getTagsOffset(), destination, destinationOffset,
+        tlen);
     }
-    return tagsIterator(cell.getTagsArray(), cell.getTagsOffset(), cell.getTagsLength());
+    return destinationOffset + tlen;
   }
 
-  private static Iterator<Tag> tagsIterator(final byte[] tags, final int offset, final int length) {
-    return new Iterator<Tag>() {
-      private int pos = offset;
-      private int endOffset = offset + length - 1;
-
-      @Override
-      public boolean hasNext() {
-        return this.pos < endOffset;
-      }
-
-      @Override
-      public Tag next() {
-        if (hasNext()) {
-          int curTagLen = Bytes.readAsInt(tags, this.pos, Tag.TAG_LENGTH_SIZE);
-          Tag tag = new ArrayBackedTag(tags, pos, curTagLen + TAG_LENGTH_SIZE);
-          this.pos += Bytes.SIZEOF_SHORT + curTagLen;
-          return tag;
-        }
-        return null;
-      }
-
-      @Override
-      public void remove() {
-        throw new UnsupportedOperationException();
-      }
-    };
+  /**
+   * Copies the tags info into the tag portion of the cell
+   * @param cell
+   * @param destination
+   * @param destinationOffset
+   * @return the position after tags
+   */
+  public static int copyTagsTo(Cell cell, ByteBuffer destination, int destinationOffset) {
+    int tlen = cell.getTagsLength();
+    if (cell instanceof ByteBufferCell) {
+      ByteBufferUtils.copyFromBufferToBuffer(((ByteBufferCell) cell).getTagsByteBuffer(),
+        destination, ((ByteBufferCell) cell).getTagsPosition(), destinationOffset, tlen);
+    } else {
+      ByteBufferUtils.copyFromArrayToBuffer(destination, destinationOffset, cell.getTagsArray(),
+        cell.getTagsOffset(), tlen);
+    }
+    return destinationOffset + tlen;
   }
 
   /**
@@ -972,7 +887,7 @@ public class PrivateCellUtil {
    * @param type Type of the Tag to retrieve
    * @return null if there is no tag of the passed in tag type
    */
-  public static Tag getTag(Cell cell, byte type) {
+  public static Optional<Tag> getTag(Cell cell, byte type) {
     boolean bufferBacked = cell instanceof ByteBufferCell;
     int length = cell.getTagsLength();
     int offset = bufferBacked ? ((ByteBufferCell) cell).getTagsPosition() : cell.getTagsOffset();
@@ -983,17 +898,65 @@ public class PrivateCellUtil {
         ByteBuffer tagsBuffer = ((ByteBufferCell) cell).getTagsByteBuffer();
         tagLen = ByteBufferUtils.readAsInt(tagsBuffer, pos, TAG_LENGTH_SIZE);
         if (ByteBufferUtils.toByte(tagsBuffer, pos + TAG_LENGTH_SIZE) == type) {
-          return new ByteBufferTag(tagsBuffer, pos, tagLen + TAG_LENGTH_SIZE);
+          return Optional.ofNullable(new ByteBufferTag(tagsBuffer, pos, tagLen + TAG_LENGTH_SIZE));
         }
       } else {
         tagLen = Bytes.readAsInt(cell.getTagsArray(), pos, TAG_LENGTH_SIZE);
         if (cell.getTagsArray()[pos + TAG_LENGTH_SIZE] == type) {
-          return new ArrayBackedTag(cell.getTagsArray(), pos, tagLen + TAG_LENGTH_SIZE);
+          return Optional
+              .ofNullable(new ArrayBackedTag(cell.getTagsArray(), pos, tagLen + TAG_LENGTH_SIZE));
         }
       }
       pos += TAG_LENGTH_SIZE + tagLen;
     }
-    return null;
+    return Optional.ofNullable(null);
+  }
+
+  /**
+   * Util method to iterate through the tags in the given cell.
+   * @param cell The Cell over which tags iterator is needed.
+   * @return iterator for the tags
+   */
+  public static Iterator<Tag> tagsIterator(final Cell cell) {
+    final int tagsLength = cell.getTagsLength();
+    // Save an object allocation where we can
+    if (tagsLength == 0) {
+      return TagUtil.EMPTY_TAGS_ITR;
+    }
+    if (cell instanceof ByteBufferCell) {
+      return tagsIterator(((ByteBufferCell) cell).getTagsByteBuffer(),
+        ((ByteBufferCell) cell).getTagsPosition(), tagsLength);
+    }
+    return CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(), cell.getTagsLength());
+  }
+
+  private static Iterator<Tag> tagsIterator(final ByteBuffer tags, final int offset,
+      final int length) {
+    return new Iterator<Tag>() {
+      private int pos = offset;
+      private int endOffset = offset + length - 1;
+
+      @Override
+      public boolean hasNext() {
+        return this.pos < endOffset;
+      }
+
+      @Override
+      public Tag next() {
+        if (hasNext()) {
+          int curTagLen = ByteBufferUtils.readAsInt(tags, this.pos, Tag.TAG_LENGTH_SIZE);
+          Tag tag = new ByteBufferTag(tags, pos, curTagLen + Tag.TAG_LENGTH_SIZE);
+          this.pos += Bytes.SIZEOF_SHORT + curTagLen;
+          return tag;
+        }
+        return null;
+      }
+
+      @Override
+      public void remove() {
+        throw new UnsupportedOperationException();
+      }
+    };
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/73e3af00/hbase-common/src/main/java/org/apache/hadoop/hbase/RawCell.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/RawCell.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/RawCell.java
new file mode 100644
index 0000000..9e25a9a
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/RawCell.java
@@ -0,0 +1,68 @@
+/**
+ * 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 java.util.List;
+import java.util.Optional;
+
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * An extended version of cell that gives more power to CPs
+ */
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
+public interface RawCell extends Cell {
+  static final int MAX_TAGS_LENGTH = (2 * Short.MAX_VALUE) + 1;
+
+  /**
+   * Allows cloning the tags in the cell to a new byte[]
+   * @return the byte[] having the tags
+   */
+  default byte[] cloneTags() {
+    return PrivateCellUtil.cloneTags(this);
+  }
+
+  /**
+   * Creates a list of tags in the current cell
+   * @return a list of tags
+   */
+  default List<Tag> getTags() {
+    return PrivateCellUtil.getTags(this);
+  }
+
+  /**
+   * Returns the specific tag of the given type
+   * @param type the type of the tag
+   * @return the specific tag if available or null
+   */
+  // TODO : Move to individual cell impl
+  default Optional<Tag> getTag(byte type) {
+    return PrivateCellUtil.getTag(this, type);
+  }
+
+  /**
+   * Check the length of tags. If it is invalid, throw IllegalArgumentException
+   * @param tagsLength the given length of tags
+   * @throws IllegalArgumentException if tagslength is invalid
+   */
+  public static void checkForTagsLength(int tagsLength) {
+    if (tagsLength > MAX_TAGS_LENGTH) {
+      throw new IllegalArgumentException("tagslength " + tagsLength + " > " + MAX_TAGS_LENGTH);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/73e3af00/hbase-common/src/main/java/org/apache/hadoop/hbase/Tag.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/Tag.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/Tag.java
index 8a25898..8709814 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/Tag.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/Tag.java
@@ -20,10 +20,12 @@
 package org.apache.hadoop.hbase;
 
 import java.nio.ByteBuffer;
+import java.util.List;
 
+import org.apache.hadoop.hbase.util.ByteBufferUtils;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.yetus.audience.InterfaceStability;
-import org.apache.hadoop.hbase.util.Bytes;
 
 /**
  * Tags are part of cells and helps to add metadata about them.
@@ -33,7 +35,7 @@ import org.apache.hadoop.hbase.util.Bytes;
  * <p>
  * See {@link TagType} for reserved tag types.
  */
-@InterfaceAudience.Private
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
 @InterfaceStability.Evolving
 public interface Tag {
 
@@ -43,6 +45,11 @@ public interface Tag {
   public static final int MAX_TAG_LENGTH = (2 * Short.MAX_VALUE) + 1 - TAG_LENGTH_SIZE;
 
   /**
+   * Custom tags if created are suggested to be above this range. So that
+   * it does not overlap with internal tag types
+   */
+  public static final byte CUSTOM_TAG_TYPE_RANGE = (byte)64;
+  /**
    * @return the tag type
    */
   byte getType();
@@ -75,4 +82,129 @@ public interface Tag {
    * @return The {@link java.nio.ByteBuffer} containing the value bytes.
    */
   ByteBuffer getValueByteBuffer();
+
+  /**
+   * Returns tag value in a new byte array. Primarily for use client-side. If server-side, use
+   * {@link Tag#getValueArray()} with appropriate {@link Tag#getValueOffset()} and
+   * {@link Tag#getValueLength()} instead to save on allocations.
+   * @param tag The Tag whose value to be returned
+   * @return tag value in a new byte array.
+   */
+  public static byte[] cloneValue(Tag tag) {
+    int tagLength = tag.getValueLength();
+    byte[] tagArr = new byte[tagLength];
+    if (tag.hasArray()) {
+      Bytes.putBytes(tagArr, 0, tag.getValueArray(), tag.getValueOffset(), tagLength);
+    } else {
+      ByteBufferUtils.copyFromBufferToArray(tagArr, tag.getValueByteBuffer(), tag.getValueOffset(),
+        0, tagLength);
+    }
+    return tagArr;
+  }
+
+  /**
+   * Converts the value bytes of the given tag into a String value
+   * @param tag The Tag
+   * @return value as String
+   */
+  public static String getValueAsString(Tag tag) {
+    if (tag.hasArray()) {
+      return Bytes.toString(tag.getValueArray(), tag.getValueOffset(), tag.getValueLength());
+    }
+    return Bytes.toString(cloneValue(tag));
+  }
+
+  /**
+   * Matches the value part of given tags
+   * @param t1 Tag to match the value
+   * @param t2 Tag to match the value
+   * @return True if values of both tags are same.
+   */
+  public static boolean matchingValue(Tag t1, Tag t2) {
+    if (t1.hasArray() && t2.hasArray()) {
+      return Bytes.equals(t1.getValueArray(), t1.getValueOffset(), t1.getValueLength(),
+        t2.getValueArray(), t2.getValueOffset(), t2.getValueLength());
+    }
+    if (t1.hasArray()) {
+      return ByteBufferUtils.equals(t2.getValueByteBuffer(), t2.getValueOffset(),
+        t2.getValueLength(), t1.getValueArray(), t1.getValueOffset(), t1.getValueLength());
+    }
+    if (t2.hasArray()) {
+      return ByteBufferUtils.equals(t1.getValueByteBuffer(), t1.getValueOffset(),
+        t1.getValueLength(), t2.getValueArray(), t2.getValueOffset(), t2.getValueLength());
+    }
+    return ByteBufferUtils.equals(t1.getValueByteBuffer(), t1.getValueOffset(), t1.getValueLength(),
+      t2.getValueByteBuffer(), t2.getValueOffset(), t2.getValueLength());
+  }
+
+  /**
+   * Copies the tag's value bytes to the given byte array
+   * @param tag The Tag
+   * @param out The byte array where to copy the Tag value.
+   * @param offset The offset within 'out' array where to copy the Tag value.
+   */
+  public static void copyValueTo(Tag tag, byte[] out, int offset) {
+    if (tag.hasArray()) {
+      Bytes.putBytes(out, offset, tag.getValueArray(), tag.getValueOffset(), tag.getValueLength());
+    } else {
+      ByteBufferUtils.copyFromBufferToArray(out, tag.getValueByteBuffer(), tag.getValueOffset(),
+        offset, tag.getValueLength());
+    }
+  }
+
+  /**
+   * Write a list of tags into a byte array
+   * @param tags The list of tags
+   * @return the serialized tag data as bytes
+   */
+  // TODO : Remove this when we move to RawCellBuilder
+  public static byte[] fromList(List<Tag> tags) {
+    if (tags == null || tags.isEmpty()) {
+      return HConstants.EMPTY_BYTE_ARRAY;
+    }
+    int length = 0;
+    for (Tag tag : tags) {
+      length += tag.getValueLength() + Tag.INFRASTRUCTURE_SIZE;
+    }
+    byte[] b = new byte[length];
+    int pos = 0;
+    int tlen;
+    for (Tag tag : tags) {
+      tlen = tag.getValueLength();
+      pos = Bytes.putAsShort(b, pos, tlen + Tag.TYPE_LENGTH_SIZE);
+      pos = Bytes.putByte(b, pos, tag.getType());
+      if (tag.hasArray()) {
+        pos = Bytes.putBytes(b, pos, tag.getValueArray(), tag.getValueOffset(), tlen);
+      } else {
+        ByteBufferUtils.copyFromBufferToArray(b, tag.getValueByteBuffer(), tag.getValueOffset(),
+          pos, tlen);
+        pos += tlen;
+      }
+    }
+    return b;
+  }
+
+  /**
+   * Converts the value bytes of the given tag into a long value
+   * @param tag The Tag
+   * @return value as long
+   */
+  public static long getValueAsLong(Tag tag) {
+    if (tag.hasArray()) {
+      return Bytes.toLong(tag.getValueArray(), tag.getValueOffset(), tag.getValueLength());
+    }
+    return ByteBufferUtils.toLong(tag.getValueByteBuffer(), tag.getValueOffset());
+  }
+
+  /**
+   * Converts the value bytes of the given tag into a byte value
+   * @param tag The Tag
+   * @return value as byte
+   */
+  public static byte getValueAsByte(Tag tag) {
+    if (tag.hasArray()) {
+      return tag.getValueArray()[tag.getValueOffset()];
+    }
+    return ByteBufferUtils.toByte(tag.getValueByteBuffer(), tag.getValueOffset());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/73e3af00/hbase-common/src/main/java/org/apache/hadoop/hbase/TagUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/TagUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/TagUtil.java
index a4962f4..6ad66ba 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/TagUtil.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/TagUtil.java
@@ -17,55 +17,24 @@
  */
 package org.apache.hadoop.hbase;
 
-import static org.apache.hadoop.hbase.Tag.TAG_LENGTH_SIZE;
-
 import java.io.IOException;
-import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 
-import org.apache.yetus.audience.InterfaceAudience;
 import org.apache.hadoop.hbase.io.util.StreamUtils;
 import org.apache.hadoop.hbase.util.ByteBufferUtils;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
+import org.apache.yetus.audience.InterfaceAudience;
 
 @InterfaceAudience.Private
 public final class TagUtil {
 
-  // If you would like to check the length of tags, please call {@link TagUtil#checkForTagsLength()}.
-  private static final int MAX_TAGS_LENGTH = (2 * Short.MAX_VALUE) + 1;
-
-  /**
-   * Private constructor to keep this class from being instantiated.
-   */
   private TagUtil(){}
 
   /**
-   * Returns tag value in a new byte array.
-   * Primarily for use client-side. If server-side, use
-   * {@link Tag#getValueArray()} with appropriate {@link Tag#getValueOffset()}
-   * and {@link Tag#getValueLength()} instead to save on allocations.
-   *
-   * @param tag The Tag whose value to be returned
-   * @return tag value in a new byte array.
-   */
-  public static byte[] cloneValue(Tag tag) {
-    int tagLength = tag.getValueLength();
-    byte[] tagArr = new byte[tagLength];
-    if (tag.hasArray()) {
-      Bytes.putBytes(tagArr, 0, tag.getValueArray(), tag.getValueOffset(), tagLength);
-    } else {
-      ByteBufferUtils.copyFromBufferToArray(tagArr, tag.getValueByteBuffer(), tag.getValueOffset(),
-          0, tagLength);
-    }
-    return tagArr;
-  }
-
-  /**
    * Creates list of tags from given byte array, expected that it is in the expected tag format.
-   *
    * @param b The byte array
    * @param offset The offset in array where tag bytes begin
    * @param length Total length of all tags bytes
@@ -75,139 +44,14 @@ public final class TagUtil {
     List<Tag> tags = new ArrayList<>();
     int pos = offset;
     while (pos < offset + length) {
-      int tagLen = Bytes.readAsInt(b, pos, TAG_LENGTH_SIZE);
-      tags.add(new ArrayBackedTag(b, pos, tagLen + TAG_LENGTH_SIZE));
-      pos += TAG_LENGTH_SIZE + tagLen;
+      int tagLen = Bytes.readAsInt(b, pos, Tag.TAG_LENGTH_SIZE);
+      tags.add(new ArrayBackedTag(b, pos, tagLen + Tag.TAG_LENGTH_SIZE));
+      pos += Tag.TAG_LENGTH_SIZE + tagLen;
     }
     return tags;
   }
 
   /**
-   * Creates list of tags from given ByteBuffer, expected that it is in the expected tag format.
-   *
-   * @param b The ByteBuffer
-   * @param offset The offset in ByteBuffer where tag bytes begin
-   * @param length Total length of all tags bytes
-   * @return List of tags
-   */
-  public static List<Tag> asList(ByteBuffer b, int offset, int length) {
-    List<Tag> tags = new ArrayList<>();
-    int pos = offset;
-    while (pos < offset + length) {
-      int tagLen = ByteBufferUtils.readAsInt(b, pos, TAG_LENGTH_SIZE);
-      tags.add(new ByteBufferTag(b, pos, tagLen + TAG_LENGTH_SIZE));
-      pos += TAG_LENGTH_SIZE + tagLen;
-    }
-    return tags;
-  }
-
-  /**
-   * Write a list of tags into a byte array
-   *
-   * @param tags The list of tags
-   * @return the serialized tag data as bytes
-   */
-  public static byte[] fromList(List<Tag> tags) {
-    if (tags == null || tags.isEmpty()) {
-      return HConstants.EMPTY_BYTE_ARRAY;
-    }
-    int length = 0;
-    for (Tag tag : tags) {
-      length += tag.getValueLength() + Tag.INFRASTRUCTURE_SIZE;
-    }
-    byte[] b = new byte[length];
-    int pos = 0;
-    int tlen;
-    for (Tag tag : tags) {
-      tlen = tag.getValueLength();
-      pos = Bytes.putAsShort(b, pos, tlen + Tag.TYPE_LENGTH_SIZE);
-      pos = Bytes.putByte(b, pos, tag.getType());
-      if (tag.hasArray()) {
-        pos = Bytes.putBytes(b, pos, tag.getValueArray(), tag.getValueOffset(), tlen);
-      } else {
-        ByteBufferUtils.copyFromBufferToArray(b, tag.getValueByteBuffer(), tag.getValueOffset(),
-            pos, tlen);
-        pos += tlen;
-      }
-    }
-    return b;
-  }
-
-  /**
-   * Converts the value bytes of the given tag into a long value
-   * @param tag The Tag
-   * @return value as long
-   */
-  public static long getValueAsLong(Tag tag) {
-    if (tag.hasArray()) {
-      return Bytes.toLong(tag.getValueArray(), tag.getValueOffset(), tag.getValueLength());
-    }
-    return ByteBufferUtils.toLong(tag.getValueByteBuffer(), tag.getValueOffset());
-  }
-
-  /**
-   * Converts the value bytes of the given tag into a byte value
-   * @param tag The Tag
-   * @return value as byte
-   */
-  public static byte getValueAsByte(Tag tag) {
-    if (tag.hasArray()) {
-      return tag.getValueArray()[tag.getValueOffset()];
-    }
-    return ByteBufferUtils.toByte(tag.getValueByteBuffer(), tag.getValueOffset());
-  }
-
-  /**
-   * Converts the value bytes of the given tag into a String value
-   * @param tag The Tag
-   * @return value as String
-   */
-  public static String getValueAsString(Tag tag){
-    if(tag.hasArray()){
-      return Bytes.toString(tag.getValueArray(), tag.getValueOffset(), tag.getValueLength());
-    }
-    return Bytes.toString(cloneValue(tag));
-  }
-
-  /**
-   * Matches the value part of given tags
-   * @param t1 Tag to match the value
-   * @param t2 Tag to match the value
-   * @return True if values of both tags are same.
-   */
-  public static boolean matchingValue(Tag t1, Tag t2) {
-    if (t1.hasArray() && t2.hasArray()) {
-      return Bytes.equals(t1.getValueArray(), t1.getValueOffset(), t1.getValueLength(),
-          t2.getValueArray(), t2.getValueOffset(), t2.getValueLength());
-    }
-    if (t1.hasArray()) {
-      return ByteBufferUtils.equals(t2.getValueByteBuffer(), t2.getValueOffset(),
-          t2.getValueLength(), t1.getValueArray(), t1.getValueOffset(), t1.getValueLength());
-    }
-    if (t2.hasArray()) {
-      return ByteBufferUtils.equals(t1.getValueByteBuffer(), t1.getValueOffset(),
-          t1.getValueLength(), t2.getValueArray(), t2.getValueOffset(), t2.getValueLength());
-    }
-    return ByteBufferUtils.equals(t1.getValueByteBuffer(), t1.getValueOffset(), t1.getValueLength(),
-        t2.getValueByteBuffer(), t2.getValueOffset(), t2.getValueLength());
-  }
-
-  /**
-   * Copies the tag's value bytes to the given byte array
-   * @param tag The Tag
-   * @param out The byte array where to copy the Tag value.
-   * @param offset The offset within 'out' array where to copy the Tag value.
-   */
-  public static void copyValueTo(Tag tag, byte[] out, int offset) {
-    if (tag.hasArray()) {
-      Bytes.putBytes(out, offset, tag.getValueArray(), tag.getValueOffset(), tag.getValueLength());
-    } else {
-      ByteBufferUtils.copyFromBufferToArray(out, tag.getValueByteBuffer(), tag.getValueOffset(),
-          offset, tag.getValueLength());
-    }
-  }
-
-  /**
    * Reads an int value stored as a VInt at tag's given offset.
    * @param tag The Tag
    * @param offset The offset where VInt bytes begin
@@ -313,16 +157,4 @@ public final class TagUtil {
       throw new UnsupportedOperationException();
     }
   };
-
-  /**
-   * Check the length of tags. If it is invalid, throw IllegalArgumentException
-   *
-   * @param tagsLength
-   * @throws IllegalArgumentException if tagslength is invalid
-   */
-  public static void checkForTagsLength(int tagsLength) {
-    if (tagsLength > MAX_TAGS_LENGTH) {
-      throw new IllegalArgumentException("tagslength "+ tagsLength + " > " + MAX_TAGS_LENGTH);
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/73e3af00/hbase-common/src/test/java/org/apache/hadoop/hbase/TestByteBufferKeyValue.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestByteBufferKeyValue.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestByteBufferKeyValue.java
index c5ce8de..f179bfe 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestByteBufferKeyValue.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestByteBufferKeyValue.java
@@ -17,7 +17,7 @@
 package org.apache.hadoop.hbase;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertFalse;
 
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
@@ -56,7 +56,7 @@ public class TestByteBufferKeyValue {
     KeyValue kvCell = new KeyValue(row1, fam1, qual1, 0l, Type.Put, row1);
     ByteBuffer buf = ByteBuffer.allocateDirect(kvCell.getBuffer().length);
     ByteBufferUtils.copyFromArrayToBuffer(buf, kvCell.getBuffer(), 0, kvCell.getBuffer().length);
-    ByteBufferCell offheapKV = new ByteBufferKeyValue(buf, 0, buf.capacity(), 0l);
+    ByteBufferCell offheapKV = new ByteBufferKeyValue(buf, 0, buf.capacity(), 0L);
     assertEquals(
       ROW1,
       ByteBufferUtils.toStringBinary(offheapKV.getRowByteBuffer(),
@@ -138,7 +138,7 @@ public class TestByteBufferKeyValue {
     KeyValue kvCell = new KeyValue(row1, fam1, qual1, 0l, Type.Put, row1, tags);
     ByteBuffer buf = ByteBuffer.allocateDirect(kvCell.getBuffer().length);
     ByteBufferUtils.copyFromArrayToBuffer(buf, kvCell.getBuffer(), 0, kvCell.getBuffer().length);
-    ByteBufferCell offheapKV = new ByteBufferKeyValue(buf, 0, buf.capacity(), 0l);
+    ByteBufferKeyValue offheapKV = new ByteBufferKeyValue(buf, 0, buf.capacity(), 0l);
     assertEquals(
       ROW1,
       ByteBufferUtils.toStringBinary(offheapKV.getRowByteBuffer(),
@@ -158,18 +158,19 @@ public class TestByteBufferKeyValue {
     assertEquals(0L, offheapKV.getTimestamp());
     assertEquals(Type.Put.getCode(), offheapKV.getTypeByte());
     // change tags to handle both onheap and offheap stuff
-    List<Tag> resTags = TagUtil.asList(offheapKV.getTagsArray(), offheapKV.getTagsOffset(),
-        offheapKV.getTagsLength());
+    List<Tag> resTags = offheapKV.getTags();
     Tag tag1 = resTags.get(0);
     assertEquals(t1.getType(), tag1.getType());
-    assertEquals(TagUtil.getValueAsString(t1), TagUtil.getValueAsString(tag1));
+    assertEquals(Tag.getValueAsString(t1),
+      Tag.getValueAsString(tag1));
     Tag tag2 = resTags.get(1);
     assertEquals(tag2.getType(), tag2.getType());
-    assertEquals(TagUtil.getValueAsString(t2), TagUtil.getValueAsString(tag2));
-    Tag res = PrivateCellUtil.getTag(offheapKV, (byte) 2);
-    assertEquals(TagUtil.getValueAsString(t2), TagUtil.getValueAsString(tag2));
-    res = PrivateCellUtil.getTag(offheapKV, (byte) 3);
-    assertNull(res);
+    assertEquals(Tag.getValueAsString(t2),
+      Tag.getValueAsString(tag2));
+    Tag res = PrivateCellUtil.getTag(offheapKV, (byte) 2).get();
+    assertEquals(Tag.getValueAsString(t2),
+      Tag.getValueAsString(tag2));
+    assertFalse(PrivateCellUtil.getTag(offheapKV, (byte) 3).isPresent());
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hbase/blob/73e3af00/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java
index 4ff4f05..45adc98 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestKeyValue.java
@@ -459,11 +459,11 @@ public class TestKeyValue extends TestCase {
     boolean meta1Ok = false, meta2Ok = false;
     for (Tag tag : tags) {
       if (tag.getType() == (byte) 1) {
-        if (Bytes.equals(TagUtil.cloneValue(tag), metaValue1)) {
+        if (Bytes.equals(Tag.cloneValue(tag), metaValue1)) {
           meta1Ok = true;
         }
       } else {
-        if (Bytes.equals(TagUtil.cloneValue(tag), metaValue2)) {
+        if (Bytes.equals(Tag.cloneValue(tag), metaValue2)) {
           meta2Ok = true;
         }
       }
@@ -476,12 +476,12 @@ public class TestKeyValue extends TestCase {
     Tag next = tagItr.next();
     assertEquals(10, next.getValueLength());
     assertEquals((byte) 1, next.getType());
-    Bytes.equals(TagUtil.cloneValue(next), metaValue1);
+    Bytes.equals(Tag.cloneValue(next), metaValue1);
     assertTrue(tagItr.hasNext());
     next = tagItr.next();
     assertEquals(10, next.getValueLength());
     assertEquals((byte) 2, next.getType());
-    Bytes.equals(TagUtil.cloneValue(next), metaValue2);
+    Bytes.equals(Tag.cloneValue(next), metaValue2);
     assertFalse(tagItr.hasNext());
 
     tagItr = PrivateCellUtil.tagsIterator(kv);
@@ -489,12 +489,12 @@ public class TestKeyValue extends TestCase {
     next = tagItr.next();
     assertEquals(10, next.getValueLength());
     assertEquals((byte) 1, next.getType());
-    Bytes.equals(TagUtil.cloneValue(next), metaValue1);
+    Bytes.equals(Tag.cloneValue(next), metaValue1);
     assertTrue(tagItr.hasNext());
     next = tagItr.next();
     assertEquals(10, next.getValueLength());
     assertEquals((byte) 2, next.getType());
-    Bytes.equals(TagUtil.cloneValue(next), metaValue2);
+    Bytes.equals(Tag.cloneValue(next), metaValue2);
     assertFalse(tagItr.hasNext());
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/73e3af00/hbase-common/src/test/java/org/apache/hadoop/hbase/TestTagUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestTagUtil.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestTagUtil.java
index d7894f4..f43a4e0 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestTagUtil.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestTagUtil.java
@@ -37,13 +37,13 @@ public class TestTagUtil {
     assertEquals(1, tags.size());
     Tag ttlTag = tags.get(0);
     assertEquals(TagType.TTL_TAG_TYPE, ttlTag.getType());
-    assertEquals(ttl, TagUtil.getValueAsLong(ttlTag));
+    assertEquals(ttl, Tag.getValueAsLong(ttlTag));
     // Already having a TTL tag in the list. So the call must remove the old tag
     long ttl2 = 30 * 1000;
     tags = TagUtil.carryForwardTTLTag(tags, ttl2);
     assertEquals(1, tags.size());
     ttlTag = tags.get(0);
     assertEquals(TagType.TTL_TAG_TYPE, ttlTag.getType());
-    assertEquals(ttl2, TagUtil.getValueAsLong(ttlTag));
+    assertEquals(ttl2, Tag.getValueAsLong(ttlTag));
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/73e3af00/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestCellCodecWithTags.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestCellCodecWithTags.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestCellCodecWithTags.java
index 1b7302f..fad30de 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestCellCodecWithTags.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestCellCodecWithTags.java
@@ -28,22 +28,21 @@ import java.io.DataOutputStream;
 import java.io.IOException;
 import java.util.List;
 
+import org.apache.hadoop.hbase.ArrayBackedTag;
 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.RawCell;
 import org.apache.hadoop.hbase.Tag;
-import org.apache.hadoop.hbase.TagUtil;
-import org.apache.hadoop.hbase.ArrayBackedTag;
+import org.apache.hadoop.hbase.shaded.com.google.common.io.CountingInputStream;
+import org.apache.hadoop.hbase.shaded.com.google.common.io.CountingOutputStream;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.io.CountingInputStream;
-import org.apache.hadoop.hbase.shaded.com.google.common.io.CountingOutputStream;
-
 @Category({MiscTests.class, SmallTests.class})
 public class TestCellCodecWithTags {
 
@@ -79,36 +78,37 @@ public class TestCellCodecWithTags {
     assertTrue(decoder.advance());
     Cell c = decoder.current();
     assertTrue(CellUtil.equals(c, cell1));
-    List<Tag> tags = TagUtil.asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength());
+    List<Tag> tags =
+        ((RawCell)c).getTags();
     assertEquals(2, tags.size());
     Tag tag = tags.get(0);
     assertEquals(1, tag.getType());
-    assertTrue(Bytes.equals(Bytes.toBytes("teststring1"), TagUtil.cloneValue(tag)));
+    assertTrue(Bytes.equals(Bytes.toBytes("teststring1"), Tag.cloneValue(tag)));
     tag = tags.get(1);
     assertEquals(2, tag.getType());
-    assertTrue(Bytes.equals(Bytes.toBytes("teststring2"), TagUtil.cloneValue(tag)));
+    assertTrue(Bytes.equals(Bytes.toBytes("teststring2"), Tag.cloneValue(tag)));
     assertTrue(decoder.advance());
     c = decoder.current();
     assertTrue(CellUtil.equals(c, cell2));
-    tags = TagUtil.asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength());
+    tags = ((RawCell)c).getTags();
     assertEquals(1, tags.size());
     tag = tags.get(0);
     assertEquals(1, tag.getType());
-    assertTrue(Bytes.equals(Bytes.toBytes("teststring3"), TagUtil.cloneValue(tag)));
+    assertTrue(Bytes.equals(Bytes.toBytes("teststring3"), Tag.cloneValue(tag)));
     assertTrue(decoder.advance());
     c = decoder.current();
     assertTrue(CellUtil.equals(c, cell3));
-    tags = TagUtil.asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength());
+    tags = ((RawCell)c).getTags();
     assertEquals(3, tags.size());
     tag = tags.get(0);
     assertEquals(2, tag.getType());
-    assertTrue(Bytes.equals(Bytes.toBytes("teststring4"), TagUtil.cloneValue(tag)));
+    assertTrue(Bytes.equals(Bytes.toBytes("teststring4"), Tag.cloneValue(tag)));
     tag = tags.get(1);
     assertEquals(2, tag.getType());
-    assertTrue(Bytes.equals(Bytes.toBytes("teststring5"), TagUtil.cloneValue(tag)));
+    assertTrue(Bytes.equals(Bytes.toBytes("teststring5"), Tag.cloneValue(tag)));
     tag = tags.get(2);
     assertEquals(1, tag.getType());
-    assertTrue(Bytes.equals(Bytes.toBytes("teststring6"), TagUtil.cloneValue(tag)));
+    assertTrue(Bytes.equals(Bytes.toBytes("teststring6"), Tag.cloneValue(tag)));
     assertFalse(decoder.advance());
     dis.close();
     assertEquals(offset, cis.getCount());

http://git-wip-us.apache.org/repos/asf/hbase/blob/73e3af00/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestKeyValueCodecWithTags.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestKeyValueCodecWithTags.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestKeyValueCodecWithTags.java
index badf048..27c6430 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestKeyValueCodecWithTags.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/codec/TestKeyValueCodecWithTags.java
@@ -28,22 +28,21 @@ import java.io.DataOutputStream;
 import java.io.IOException;
 import java.util.List;
 
+import org.apache.hadoop.hbase.ArrayBackedTag;
 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.RawCell;
 import org.apache.hadoop.hbase.Tag;
-import org.apache.hadoop.hbase.TagUtil;
-import org.apache.hadoop.hbase.ArrayBackedTag;
+import org.apache.hadoop.hbase.shaded.com.google.common.io.CountingInputStream;
+import org.apache.hadoop.hbase.shaded.com.google.common.io.CountingOutputStream;
 import org.apache.hadoop.hbase.testclassification.MiscTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.io.CountingInputStream;
-import org.apache.hadoop.hbase.shaded.com.google.common.io.CountingOutputStream;
-
 @Category({MiscTests.class, SmallTests.class})
 public class TestKeyValueCodecWithTags {
 
@@ -79,36 +78,37 @@ public class TestKeyValueCodecWithTags {
     assertTrue(decoder.advance());
     Cell c = decoder.current();
     assertTrue(CellUtil.equals(c, kv1));
-    List<Tag> tags = TagUtil.asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength());
+    List<Tag> tags =
+        ((RawCell)c).getTags();
     assertEquals(2, tags.size());
     Tag tag = tags.get(0);
     assertEquals(1, tag.getType());
-    assertTrue(Bytes.equals(Bytes.toBytes("teststring1"), TagUtil.cloneValue(tag)));
+    assertTrue(Bytes.equals(Bytes.toBytes("teststring1"), Tag.cloneValue(tag)));
     tag = tags.get(1);
     assertEquals(2, tag.getType());
-    assertTrue(Bytes.equals(Bytes.toBytes("teststring2"), TagUtil.cloneValue(tag)));
+    assertTrue(Bytes.equals(Bytes.toBytes("teststring2"), Tag.cloneValue(tag)));
     assertTrue(decoder.advance());
     c = decoder.current();
     assertTrue(CellUtil.equals(c, kv2));
-    tags = TagUtil.asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength());
+    tags = ((RawCell)c).getTags();
     assertEquals(1, tags.size());
     tag = tags.get(0);
     assertEquals(1, tag.getType());
-    assertTrue(Bytes.equals(Bytes.toBytes("teststring3"), TagUtil.cloneValue(tag)));
+    assertTrue(Bytes.equals(Bytes.toBytes("teststring3"), Tag.cloneValue(tag)));
     assertTrue(decoder.advance());
     c = decoder.current();
     assertTrue(CellUtil.equals(c, kv3));
-    tags = TagUtil.asList(c.getTagsArray(), c.getTagsOffset(), c.getTagsLength());
+    tags = ((RawCell)c).getTags();
     assertEquals(3, tags.size());
     tag = tags.get(0);
     assertEquals(2, tag.getType());
-    assertTrue(Bytes.equals(Bytes.toBytes("teststring4"), TagUtil.cloneValue(tag)));
+    assertTrue(Bytes.equals(Bytes.toBytes("teststring4"), Tag.cloneValue(tag)));
     tag = tags.get(1);
     assertEquals(2, tag.getType());
-    assertTrue(Bytes.equals(Bytes.toBytes("teststring5"), TagUtil.cloneValue(tag)));
+    assertTrue(Bytes.equals(Bytes.toBytes("teststring5"), Tag.cloneValue(tag)));
     tag = tags.get(2);
     assertEquals(1, tag.getType());
-    assertTrue(Bytes.equals(Bytes.toBytes("teststring6"), TagUtil.cloneValue(tag)));
+    assertTrue(Bytes.equals(Bytes.toBytes("teststring6"), Tag.cloneValue(tag)));
     assertFalse(decoder.advance());
     dis.close();
     assertEquals(offset, cis.getCount());

http://git-wip-us.apache.org/repos/asf/hbase/blob/73e3af00/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
index ffe3fe1..aa2984f 100644
--- a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHFileOutputFormat2.java
@@ -53,6 +53,7 @@ import org.apache.hadoop.hbase.CategoryBasedTimeout;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
+import org.apache.hadoop.hbase.RawCell;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
@@ -65,7 +66,6 @@ import org.apache.hadoop.hbase.PerformanceEvaluation;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.TagType;
-import org.apache.hadoop.hbase.TagUtil;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
@@ -491,8 +491,7 @@ public class TestHFileOutputFormat2  {
         HFileScanner scanner = reader.getScanner(false, false, false);
         scanner.seekTo();
         Cell cell = scanner.getCell();
-        List<Tag> tagsFromCell = TagUtil.asList(cell.getTagsArray(), cell.getTagsOffset(),
-            cell.getTagsLength());
+        List<Tag> tagsFromCell = ((RawCell)cell).getTags();
         assertTrue(tagsFromCell.size() > 0);
         for (Tag tag : tagsFromCell) {
           assertTrue(tag.getType() == TagType.TTL_TAG_TYPE);

http://git-wip-us.apache.org/repos/asf/hbase/blob/73e3af00/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionCoprocessorEnvironment.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionCoprocessorEnvironment.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionCoprocessorEnvironment.java
index 1f48824..ca57fc8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionCoprocessorEnvironment.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionCoprocessorEnvironment.java
@@ -24,6 +24,7 @@ import java.util.concurrent.ConcurrentMap;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
+import org.apache.hadoop.hbase.ExtendedCellBuilder;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.Connection;
@@ -103,4 +104,11 @@ public interface RegionCoprocessorEnvironment extends CoprocessorEnvironment<Reg
   // so we do not want to allow coprocessors to export metrics at the region level. We can allow
   // getMetricRegistryForTable() to allow coprocessors to track metrics per-table, per-regionserver.
   MetricRegistry getMetricRegistryForRegionServer();
+
+  /**
+   * Returns a CellBuilder so that coprocessors can build cells. These cells can also include tags.
+   * Note that this builder does not support updating seqId of the cells
+   * @return the ExtendedCellBuilder
+   */
+  ExtendedCellBuilder getCellBuilder();
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/73e3af00/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java
index 990ac5e..5aea107 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFilePrettyPrinter.java
@@ -55,22 +55,20 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.RawCell;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
-import org.apache.hadoop.hbase.PrivateCellUtil;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
+import org.apache.hadoop.hbase.PrivateCellUtil;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Tag;
-import org.apache.hadoop.hbase.TagUtil;
-import org.apache.hadoop.hbase.regionserver.HStoreFile;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
 import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
 import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
 import org.apache.hadoop.hbase.mob.MobUtils;
+import org.apache.hadoop.hbase.regionserver.HStoreFile;
 import org.apache.hadoop.hbase.regionserver.TimeRangeTracker;
 import org.apache.hadoop.hbase.util.BloomFilter;
 import org.apache.hadoop.hbase.util.BloomFilterFactory;
@@ -80,6 +78,8 @@ import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HFileArchiveUtil;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.yetus.audience.InterfaceStability;
 
 import com.codahale.metrics.ConsoleReporter;
 import com.codahale.metrics.Counter;
@@ -399,8 +399,7 @@ public class HFilePrettyPrinter extends Configured implements Tool {
               + Bytes.toStringBinary(cell.getValueArray(), cell.getValueOffset(),
                   cell.getValueLength()));
           int i = 0;
-          List<Tag> tags = TagUtil.asList(cell.getTagsArray(), cell.getTagsOffset(),
-              cell.getTagsLength());
+          List<Tag> tags = ((RawCell)cell).getTags();
           for (Tag tag : tags) {
             out.print(String.format(" T[%d]: %s", i++, tag.toString()));
           }
@@ -442,7 +441,7 @@ public class HFilePrettyPrinter extends Configured implements Tool {
           System.err.println("ERROR, wrong value format in mob reference cell "
             + CellUtil.getCellKeyAsString(cell));
         } else {
-          TableName tn = TableName.valueOf(TagUtil.cloneValue(tnTag));
+          TableName tn = TableName.valueOf(Tag.cloneValue(tnTag));
           String mobFileName = MobUtils.getMobFileName(cell);
           boolean exist = mobFileExists(fs, tn, mobFileName,
             Bytes.toString(CellUtil.cloneFamily(cell)), foundMobFiles, missingMobFiles);

http://git-wip-us.apache.org/repos/asf/hbase/blob/73e3af00/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java
index 8407783..f77df4c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobUtils.java
@@ -27,6 +27,7 @@ import java.util.Calendar;
 import java.util.Collection;
 import java.util.Date;
 import java.util.List;
+import java.util.Optional;
 import java.util.UUID;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.RejectedExecutionException;
@@ -101,7 +102,7 @@ public final class MobUtils {
   static {
     List<Tag> tags = new ArrayList<>();
     tags.add(MobConstants.MOB_REF_TAG);
-    REF_DELETE_MARKER_TAG_BYTES = TagUtil.fromList(tags);
+    REF_DELETE_MARKER_TAG_BYTES = Tag.fromList(tags);
   }
 
   /**
@@ -175,8 +176,10 @@ public final class MobUtils {
    */
   public static boolean isMobReferenceCell(Cell cell) {
     if (cell.getTagsLength() > 0) {
-      Tag tag = PrivateCellUtil.getTag(cell, TagType.MOB_REFERENCE_TAG_TYPE);
-      return tag != null;
+      Optional<Tag> tag = PrivateCellUtil.getTag(cell, TagType.MOB_REFERENCE_TAG_TYPE);
+      if (tag.isPresent()) {
+        return true;
+      }
     }
     return false;
   }
@@ -188,7 +191,10 @@ public final class MobUtils {
    */
   public static Tag getTableNameTag(Cell cell) {
     if (cell.getTagsLength() > 0) {
-      return PrivateCellUtil.getTag(cell, TagType.MOB_TABLE_NAME_TAG_TYPE);
+      Optional<Tag> tag = PrivateCellUtil.getTag(cell, TagType.MOB_TABLE_NAME_TAG_TYPE);
+      if (tag.isPresent()) {
+        return tag.get();
+      }
     }
     return null;
   }
@@ -496,7 +502,7 @@ public final class MobUtils {
     // find the original mob files by this table name. For details please see cloning
     // snapshot for mob files.
     tags.add(tableNameTag);
-    return createMobRefCell(cell, fileName, TagUtil.fromList(tags));
+    return createMobRefCell(cell, fileName, Tag.fromList(tags));
   }
 
   public static Cell createMobRefCell(Cell cell, byte[] fileName, byte[] refCellTags) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/73e3af00/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java
index 3064723..0cccfa3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/compactions/PartitionedMobCompactor.java
@@ -54,7 +54,6 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.TagType;
-import org.apache.hadoop.hbase.TagUtil;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
@@ -84,14 +83,13 @@ import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
 import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
 import org.apache.hadoop.hbase.regionserver.StoreScanner;
 import org.apache.hadoop.hbase.security.EncryptionUtil;
+import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.yetus.audience.InterfaceAudience;
 
-import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
-
 /**
  * An implementation of {@link MobCompactor} that compacts the mob files in partitions.
  */
@@ -133,7 +131,7 @@ public class PartitionedMobCompactor extends MobCompactor {
     tags.add(MobConstants.MOB_REF_TAG);
     Tag tableNameTag = new ArrayBackedTag(TagType.MOB_TABLE_NAME_TAG_TYPE, tableName.getName());
     tags.add(tableNameTag);
-    this.refCellTags = TagUtil.fromList(tags);
+    this.refCellTags = Tag.fromList(tags);
     cryptoContext = EncryptionUtil.createEncryptionContext(copyOfConf, column);
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/73e3af00/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java
index 5cb1e45..5db7383 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HMobStore.java
@@ -42,7 +42,6 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.TagType;
-import org.apache.hadoop.hbase.TagUtil;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.filter.Filter;
@@ -121,7 +120,7 @@ public class HMobStore extends HStore {
     Tag tableNameTag = new ArrayBackedTag(TagType.MOB_TABLE_NAME_TAG_TYPE,
         getTableName().getName());
     tags.add(tableNameTag);
-    this.refCellTags = TagUtil.fromList(tags);
+    this.refCellTags = Tag.fromList(tags);
   }
 
   /**
@@ -332,7 +331,7 @@ public class HMobStore extends HStore {
       String fileName = MobUtils.getMobFileName(reference);
       Tag tableNameTag = MobUtils.getTableNameTag(reference);
       if (tableNameTag != null) {
-        String tableNameString = TagUtil.getValueAsString(tableNameTag);
+        String tableNameString = Tag.getValueAsString(tableNameTag);
         List<Path> locations = map.get(tableNameString);
         if (locations == null) {
           IdLock.Entry lockEntry = keyLock.getLockEntry(tableNameString.hashCode());
@@ -359,12 +358,15 @@ public class HMobStore extends HStore {
           + "qualifier,timestamp,type and tags but with an empty value to return.");
       result = ExtendedCellBuilderFactory.create(CellBuilderType.DEEP_COPY)
               .setRow(reference.getRowArray(), reference.getRowOffset(), reference.getRowLength())
-              .setFamily(reference.getFamilyArray(), reference.getFamilyOffset(), reference.getFamilyLength())
-              .setQualifier(reference.getQualifierArray(), reference.getQualifierOffset(), reference.getQualifierLength())
+              .setFamily(reference.getFamilyArray(), reference.getFamilyOffset(),
+                reference.getFamilyLength())
+              .setQualifier(reference.getQualifierArray(),
+                reference.getQualifierOffset(), reference.getQualifierLength())
               .setTimestamp(reference.getTimestamp())
               .setType(reference.getTypeByte())
               .setValue(HConstants.EMPTY_BYTE_ARRAY)
-              .setTags(reference.getTagsArray(), reference.getTagsOffset(), reference.getTagsLength())
+              .setTags(reference.getTagsArray(), reference.getTagsOffset(),
+                reference.getTagsLength())
               .build();
     }
     return result;

http://git-wip-us.apache.org/repos/asf/hbase/blob/73e3af00/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index cc678d2..c767dc5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -7714,7 +7714,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
               .setTimestamp(Math.max(currentCell.getTimestamp() + 1, now))
               .setType(KeyValue.Type.Put.getCode())
               .setValue(newValue, 0, newValue.length)
-              .setTags(TagUtil.fromList(tags))
+              .setTags(Tag.fromList(tags))
               .build();
     } else {
       PrivateCellUtil.updateLatestStamp(delta, now);