You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by sy...@apache.org on 2016/01/07 17:58:46 UTC

[14/17] hbase git commit: HBASE-12593 Tags to work with ByteBuffer.

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/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 faf6d81..496c7e2 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
@@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.TagUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.filter.Filter;
@@ -338,8 +339,7 @@ public class HMobStore extends HStore {
       String fileName = MobUtils.getMobFileName(reference);
       Tag tableNameTag = MobUtils.getTableNameTag(reference);
       if (tableNameTag != null) {
-        byte[] tableName = tableNameTag.getValue();
-        String tableNameString = Bytes.toString(tableName);
+        String tableNameString = TagUtil.getValueAsString(tableNameTag);
         List<Path> locations = map.get(tableNameString);
         if (locations == null) {
           IdLock.Entry lockEntry = keyLock.getLockEntry(tableNameString.hashCode());
@@ -347,7 +347,7 @@ public class HMobStore extends HStore {
             locations = map.get(tableNameString);
             if (locations == null) {
               locations = new ArrayList<Path>(2);
-              TableName tn = TableName.valueOf(tableName);
+              TableName tn = TableName.valueOf(tableNameString);
               locations.add(MobUtils.getMobFamilyPath(conf, tn, family.getNameAsString()));
               locations.add(HFileArchiveUtil.getStoreArchivePath(conf, tn, MobUtils
                   .getMobRegionInfo(tn).getEncodedName(), family.getNameAsString()));

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/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 ccf2eb0..e553fcc 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
@@ -69,6 +69,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellScanner;
@@ -94,6 +95,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.TagRewriteCell;
 import org.apache.hadoop.hbase.TagType;
+import org.apache.hadoop.hbase.TagUtil;
 import org.apache.hadoop.hbase.UnknownScannerException;
 import org.apache.hadoop.hbase.backup.HFileArchiver;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
@@ -3667,8 +3669,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       for (int i = 0; i < listSize; i++) {
         Cell cell = cells.get(i);
         List<Tag> newTags = new ArrayList<Tag>();
-        Iterator<Tag> tagIterator = CellUtil.tagsIterator(cell.getTagsArray(),
-          cell.getTagsOffset(), cell.getTagsLength());
+        Iterator<Tag> tagIterator = CellUtil.tagsIterator(cell);
 
         // Carry forward existing tags
 
@@ -3685,11 +3686,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         // above may change when there are more tag based features in core.
         if (m.getTTL() != Long.MAX_VALUE) {
           // Add a cell TTL tag
-          newTags.add(new Tag(TagType.TTL_TAG_TYPE, Bytes.toBytes(m.getTTL())));
+          newTags.add(new ArrayBackedTag(TagType.TTL_TAG_TYPE, Bytes.toBytes(m.getTTL())));
         }
 
         // Rewrite the cell with the updated set of tags
-        cells.set(i, new TagRewriteCell(cell, Tag.fromList(newTags)));
+        cells.set(i, new TagRewriteCell(cell, TagUtil.fromList(newTags)));
       }
     }
   }
@@ -7073,8 +7074,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   private static List<Tag> carryForwardTags(final Cell cell, final List<Tag> tags) {
     if (cell.getTagsLength() <= 0) return tags;
     List<Tag> newTags = tags == null? new ArrayList<Tag>(): /*Append Tags*/tags; 
-    Iterator<Tag> i =
-        CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(), cell.getTagsLength());
+    Iterator<Tag> i = CellUtil.tagsIterator(cell);
     while (i.hasNext()) newTags.add(i.next());
     return newTags;
   }
@@ -7178,11 +7178,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
 
                 if (mutate.getTTL() != Long.MAX_VALUE) {
                   // Add the new TTL tag
-                  newTags.add(new Tag(TagType.TTL_TAG_TYPE, Bytes.toBytes(mutate.getTTL())));
+                  newTags.add(
+                      new ArrayBackedTag(TagType.TTL_TAG_TYPE, Bytes.toBytes(mutate.getTTL())));
                 }
 
                 // Rebuild tags
-                byte[] tagBytes = Tag.fromList(newTags);
+                byte[] tagBytes = TagUtil.fromList(newTags);
 
                 // allocate an empty cell once
                 newCell = new KeyValue(row.length, cell.getFamilyLength(),
@@ -7216,9 +7217,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
 
                 if (mutate.getTTL() != Long.MAX_VALUE) {
                   List<Tag> newTags = new ArrayList<Tag>(1);
-                  newTags.add(new Tag(TagType.TTL_TAG_TYPE, Bytes.toBytes(mutate.getTTL())));
+                  newTags.add(
+                      new ArrayBackedTag(TagType.TTL_TAG_TYPE, Bytes.toBytes(mutate.getTTL())));
                   // Add the new TTL tag
-                  newCell = new TagRewriteCell(cell, Tag.fromList(newTags));
+                  newCell = new TagRewriteCell(cell, TagUtil.fromList(newTags));
                 } else {
                   newCell = cell;
                 }
@@ -7439,7 +7441,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
 
               // Add the TTL tag if the mutation carried one
               if (mutation.getTTL() != Long.MAX_VALUE) {
-                newTags.add(new Tag(TagType.TTL_TAG_TYPE, Bytes.toBytes(mutation.getTTL())));
+                newTags.add(
+                    new ArrayBackedTag(TagType.TTL_TAG_TYPE, Bytes.toBytes(mutation.getTTL())));
               }
 
               Cell newKV = new KeyValue(row, 0, row.length,

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
index badbd65..8d66696 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
@@ -59,6 +59,7 @@ import org.apache.hadoop.hbase.KeyValue;
 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.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.conf.ConfigurationManager;
@@ -1779,28 +1780,24 @@ public class HStore implements Store {
    * @return true if the cell is expired
    */
   static boolean isCellTTLExpired(final Cell cell, final long oldestTimestamp, final long now) {
-    // Do not create an Iterator or Tag objects unless the cell actually has tags.
-    if (cell.getTagsLength() > 0) {
-      // Look for a TTL tag first. Use it instead of the family setting if
-      // found. If a cell has multiple TTLs, resolve the conflict by using the
-      // first tag encountered.
-      Iterator<Tag> i = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(),
-        cell.getTagsLength());
-      while (i.hasNext()) {
-        Tag t = i.next();
-        if (TagType.TTL_TAG_TYPE == t.getType()) {
-          // Unlike in schema cell TTLs are stored in milliseconds, no need
-          // to convert
-          long ts = cell.getTimestamp();
-          assert t.getTagLength() == Bytes.SIZEOF_LONG;
-          long ttl = Bytes.toLong(t.getBuffer(), t.getTagOffset(), t.getTagLength());
-          if (ts + ttl < now) {
-            return true;
-          }
-          // Per cell TTLs cannot extend lifetime beyond family settings, so
-          // fall through to check that
-          break;
+    // Look for a TTL tag first. Use it instead of the family setting if
+    // found. If a cell has multiple TTLs, resolve the conflict by using the
+    // first tag encountered.
+    Iterator<Tag> i = CellUtil.tagsIterator(cell);
+    while (i.hasNext()) {
+      Tag t = i.next();
+      if (TagType.TTL_TAG_TYPE == t.getType()) {
+        // Unlike in schema cell TTLs are stored in milliseconds, no need
+        // to convert
+        long ts = cell.getTimestamp();
+        assert t.getValueLength() == Bytes.SIZEOF_LONG;
+        long ttl = TagUtil.getValueAsLong(t);
+        if (ts + ttl < now) {
+          return true;
         }
+        // Per cell TTLs cannot extend lifetime beyond family settings, so
+        // fall through to check that
+        break;
       }
     }
     return false;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
index 887af0a..f0723c2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessControlLists.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.NamespaceDescriptor;
 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.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
@@ -658,8 +659,7 @@ public class AccessControlLists {
        return null;
      }
      List<Permission> results = Lists.newArrayList();
-     Iterator<Tag> tagsIterator = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(),
-        cell.getTagsLength());
+     Iterator<Tag> tagsIterator = CellUtil.tagsIterator(cell);
      while (tagsIterator.hasNext()) {
        Tag tag = tagsIterator.next();
        if (tag.getType() == ACL_TAG_TYPE) {
@@ -668,7 +668,12 @@ public class AccessControlLists {
          // use the builder
          AccessControlProtos.UsersAndPermissions.Builder builder = 
            AccessControlProtos.UsersAndPermissions.newBuilder();
-         ProtobufUtil.mergeFrom(builder, tag.getBuffer(), tag.getTagOffset(), tag.getTagLength());
+         if (tag.hasArray()) {
+           ProtobufUtil.mergeFrom(builder, tag.getValueArray(), tag.getValueOffset(),
+               tag.getValueLength());
+         } else {
+           ProtobufUtil.mergeFrom(builder,TagUtil.cloneValue(tag));
+         }
          ListMultimap<String,Permission> kvPerms =
            ProtobufUtil.toUsersAndPermissions(builder.build());
          // Are there permissions for this user?

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
index 0d8b261..bb348a3 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.security.access;
 import java.io.IOException;
 import java.net.InetAddress;
 import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -34,6 +35,7 @@ import java.util.TreeSet;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.CellUtil;
@@ -54,6 +56,7 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.TagRewriteCell;
+import org.apache.hadoop.hbase.TagUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Append;
 import org.apache.hadoop.hbase.client.Delete;
@@ -882,15 +885,13 @@ public class AccessController extends BaseMasterAndRegionObserver
       List<Cell> newCells = Lists.newArrayList();
       for (Cell cell: e.getValue()) {
         // Prepend the supplied perms in a new ACL tag to an update list of tags for the cell
-        List<Tag> tags = Lists.newArrayList(new Tag(AccessControlLists.ACL_TAG_TYPE, perms));
-        if (cell.getTagsLength() > 0) {
-          Iterator<Tag> tagIterator = CellUtil.tagsIterator(cell.getTagsArray(),
-            cell.getTagsOffset(), cell.getTagsLength());
-          while (tagIterator.hasNext()) {
-            tags.add(tagIterator.next());
-          }
+        List<Tag> tags = new ArrayList<Tag>();
+        tags.add(new ArrayBackedTag(AccessControlLists.ACL_TAG_TYPE, perms));
+        Iterator<Tag> tagIterator = CellUtil.tagsIterator(cell);
+        while (tagIterator.hasNext()) {
+          tags.add(tagIterator.next());
         }
-        newCells.add(new TagRewriteCell(cell, Tag.fromList(tags)));
+        newCells.add(new TagRewriteCell(cell, TagUtil.fromList(tags)));
       }
       // This is supposed to be safe, won't CME
       e.setValue(newCells);
@@ -915,14 +916,10 @@ public class AccessController extends BaseMasterAndRegionObserver
       return;
     }
     for (CellScanner cellScanner = m.cellScanner(); cellScanner.advance();) {
-      Cell cell = cellScanner.current();
-      if (cell.getTagsLength() > 0) {
-        Iterator<Tag> tagsItr = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(),
-          cell.getTagsLength());
-        while (tagsItr.hasNext()) {
-          if (tagsItr.next().getType() == AccessControlLists.ACL_TAG_TYPE) {
-            throw new AccessDeniedException("Mutation contains cell with reserved type tag");
-          }
+      Iterator<Tag> tagsItr = CellUtil.tagsIterator(cellScanner.current());
+      while (tagsItr.hasNext()) {
+        if (tagsItr.next().getType() == AccessControlLists.ACL_TAG_TYPE) {
+          throw new AccessDeniedException("Mutation contains cell with reserved type tag");
         }
       }
     }
@@ -1997,32 +1994,21 @@ public class AccessController extends BaseMasterAndRegionObserver
 
     // Collect any ACLs from the old cell
     List<Tag> tags = Lists.newArrayList();
+    List<Tag> aclTags = Lists.newArrayList();
     ListMultimap<String,Permission> perms = ArrayListMultimap.create();
     if (oldCell != null) {
-      // Save an object allocation where we can
-      if (oldCell.getTagsLength() > 0) {
-        Iterator<Tag> tagIterator = CellUtil.tagsIterator(oldCell.getTagsArray(),
-          oldCell.getTagsOffset(), oldCell.getTagsLength());
-        while (tagIterator.hasNext()) {
-          Tag tag = tagIterator.next();
-          if (tag.getType() != AccessControlLists.ACL_TAG_TYPE) {
-            // Not an ACL tag, just carry it through
-            if (LOG.isTraceEnabled()) {
-              LOG.trace("Carrying forward tag from " + oldCell + ": type " + tag.getType() +
-                " length " + tag.getTagLength());
-            }
-            tags.add(tag);
-          } else {
-            // Merge the perms from the older ACL into the current permission set
-            // TODO: The efficiency of this can be improved. Don't build just to unpack
-            // again, use the builder
-            AccessControlProtos.UsersAndPermissions.Builder builder =
-              AccessControlProtos.UsersAndPermissions.newBuilder();
-            ProtobufUtil.mergeFrom(builder, tag.getBuffer(), tag.getTagOffset(), tag.getTagLength());
-            ListMultimap<String,Permission> kvPerms =
-              ProtobufUtil.toUsersAndPermissions(builder.build());
-            perms.putAll(kvPerms);
+      Iterator<Tag> tagIterator = CellUtil.tagsIterator(oldCell);
+      while (tagIterator.hasNext()) {
+        Tag tag = tagIterator.next();
+        if (tag.getType() != AccessControlLists.ACL_TAG_TYPE) {
+          // Not an ACL tag, just carry it through
+          if (LOG.isTraceEnabled()) {
+            LOG.trace("Carrying forward tag from " + oldCell + ": type " + tag.getType()
+                + " length " + tag.getValueLength());
           }
+          tags.add(tag);
+        } else {
+          aclTags.add(tag);
         }
       }
     }
@@ -2031,7 +2017,7 @@ public class AccessController extends BaseMasterAndRegionObserver
     byte[] aclBytes = mutation.getACL();
     if (aclBytes != null) {
       // Yes, use it
-      tags.add(new Tag(AccessControlLists.ACL_TAG_TYPE, aclBytes));
+      tags.add(new ArrayBackedTag(AccessControlLists.ACL_TAG_TYPE, aclBytes));
     } else {
       // No, use what we carried forward
       if (perms != null) {
@@ -2041,8 +2027,7 @@ public class AccessController extends BaseMasterAndRegionObserver
         if (LOG.isTraceEnabled()) {
           LOG.trace("Carrying forward ACLs from " + oldCell + ": " + perms);
         }
-        tags.add(new Tag(AccessControlLists.ACL_TAG_TYPE,
-            ProtobufUtil.toUsersAndPermissions(perms).toByteArray()));
+        tags.addAll(aclTags);
       }
     }
 
@@ -2051,7 +2036,7 @@ public class AccessController extends BaseMasterAndRegionObserver
       return newCell;
     }
 
-    Cell rewriteCell = new TagRewriteCell(newCell, Tag.fromList(tags));
+    Cell rewriteCell = new TagRewriteCell(newCell, TagUtil.fromList(tags));
     return rewriteCell;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/DefaultVisibilityLabelServiceImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/DefaultVisibilityLabelServiceImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/DefaultVisibilityLabelServiceImpl.java
index 42d6a03..f1aec09 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/DefaultVisibilityLabelServiceImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/DefaultVisibilityLabelServiceImpl.java
@@ -42,6 +42,7 @@ import java.util.regex.Pattern;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.AuthUtil;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
@@ -49,6 +50,7 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.TagType;
+import org.apache.hadoop.hbase.TagUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Mutation;
@@ -90,7 +92,7 @@ public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService
     } catch (IOException e) {
       // We write to a byte array. No Exception can happen.
     }
-    LABELS_TABLE_TAGS[0] = new Tag(VISIBILITY_TAG_TYPE, baos.toByteArray());
+    LABELS_TABLE_TAGS[0] = new ArrayBackedTag(VISIBILITY_TAG_TYPE, baos.toByteArray());
   }
 
   public DefaultVisibilityLabelServiceImpl() {
@@ -481,42 +483,37 @@ public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService
       @Override
       public boolean evaluate(Cell cell) throws IOException {
         boolean visibilityTagPresent = false;
-        // Save an object allocation where we can
-        if (cell.getTagsLength() > 0) {
-          Iterator<Tag> tagsItr = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(),
-              cell.getTagsLength());
-          while (tagsItr.hasNext()) {
-            boolean includeKV = true;
-            Tag tag = tagsItr.next();
-            if (tag.getType() == VISIBILITY_TAG_TYPE) {
-              visibilityTagPresent = true;
-              int offset = tag.getTagOffset();
-              int endOffset = offset + tag.getTagLength();
-              while (offset < endOffset) {
-                Pair<Integer, Integer> result = StreamUtils
-                    .readRawVarint32(tag.getBuffer(), offset);
-                int currLabelOrdinal = result.getFirst();
-                if (currLabelOrdinal < 0) {
-                  // check for the absence of this label in the Scan Auth labels
-                  // ie. to check BitSet corresponding bit is 0
-                  int temp = -currLabelOrdinal;
-                  if (bs.get(temp)) {
-                    includeKV = false;
-                    break;
-                  }
-                } else {
-                  if (!bs.get(currLabelOrdinal)) {
-                    includeKV = false;
-                    break;
-                  }
+        Iterator<Tag> tagsItr = CellUtil.tagsIterator(cell);
+        while (tagsItr.hasNext()) {
+          boolean includeKV = true;
+          Tag tag = tagsItr.next();
+          if (tag.getType() == VISIBILITY_TAG_TYPE) {
+            visibilityTagPresent = true;
+            int offset = tag.getValueOffset();
+            int endOffset = offset + tag.getValueLength();
+            while (offset < endOffset) {
+              Pair<Integer, Integer> result = TagUtil.readVIntValuePart(tag, offset);
+              int currLabelOrdinal = result.getFirst();
+              if (currLabelOrdinal < 0) {
+                // check for the absence of this label in the Scan Auth labels
+                // ie. to check BitSet corresponding bit is 0
+                int temp = -currLabelOrdinal;
+                if (bs.get(temp)) {
+                  includeKV = false;
+                  break;
+                }
+              } else {
+                if (!bs.get(currLabelOrdinal)) {
+                  includeKV = false;
+                  break;
                 }
-                offset += result.getSecond();
-              }
-              if (includeKV) {
-                // We got one visibility expression getting evaluated to true. Good to include this
-                // KV in the result then.
-                return true;
               }
+              offset += result.getSecond();
+            }
+            if (includeKV) {
+              // We got one visibility expression getting evaluated to true. Good to include this
+              // KV in the result then.
+              return true;
             }
           }
         }
@@ -596,8 +593,7 @@ public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService
       for (Tag tag : deleteVisTags) {
         matchFound = false;
         for (Tag givenTag : putVisTags) {
-          if (Bytes.equals(tag.getBuffer(), tag.getTagOffset(), tag.getTagLength(),
-              givenTag.getBuffer(), givenTag.getTagOffset(), givenTag.getTagLength())) {
+          if (TagUtil.matchingValue(tag, givenTag)) {
             matchFound = true;
             break;
           }
@@ -621,10 +617,10 @@ public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService
   private static void getSortedTagOrdinals(List<List<Integer>> fullTagsList, Tag tag)
       throws IOException {
     List<Integer> tagsOrdinalInSortedOrder = new ArrayList<Integer>();
-    int offset = tag.getTagOffset();
-    int endOffset = offset + tag.getTagLength();
+    int offset = tag.getValueOffset();
+    int endOffset = offset + tag.getValueLength();
     while (offset < endOffset) {
-      Pair<Integer, Integer> result = StreamUtils.readRawVarint32(tag.getBuffer(), offset);
+      Pair<Integer, Integer> result = TagUtil.readVIntValuePart(tag, offset);
       tagsOrdinalInSortedOrder.add(result.getFirst());
       offset += result.getSecond();
     }
@@ -678,11 +674,11 @@ public class DefaultVisibilityLabelServiceImpl implements VisibilityLabelService
           visibilityString.append(VisibilityConstants.CLOSED_PARAN).append(
               VisibilityConstants.OR_OPERATOR);
         }
-        int offset = tag.getTagOffset();
-        int endOffset = offset + tag.getTagLength();
+        int offset = tag.getValueOffset();
+        int endOffset = offset + tag.getValueLength();
         boolean expressionStart = true;
         while (offset < endOffset) {
-          Pair<Integer, Integer> result = StreamUtils.readRawVarint32(tag.getBuffer(), offset);
+          Pair<Integer, Integer> result = TagUtil.readVIntValuePart(tag, offset);
           int currLabelOrdinal = result.getFirst();
           if (currLabelOrdinal < 0) {
             int temp = -currLabelOrdinal;

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
index 5b8bdb3..b025758 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
@@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.TagRewriteCell;
 import org.apache.hadoop.hbase.TagType;
+import org.apache.hadoop.hbase.TagUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.Append;
 import org.apache.hadoop.hbase.client.Delete;
@@ -340,8 +341,7 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
           Tag tag = pair.getSecond();
           if (cellVisibility == null && tag != null) {
             // May need to store only the first one
-            cellVisibility = new CellVisibility(Bytes.toString(tag.getBuffer(), tag.getTagOffset(),
-                tag.getTagLength()));
+            cellVisibility = new CellVisibility(TagUtil.getValueAsString(tag));
             modifiedTagFound = true;
           }
         }
@@ -368,14 +368,13 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
             List<Cell> updatedCells = new ArrayList<Cell>();
             for (CellScanner cellScanner = m.cellScanner(); cellScanner.advance();) {
               Cell cell = cellScanner.current();
-              List<Tag> tags = Tag.asList(cell.getTagsArray(), cell.getTagsOffset(),
-                  cell.getTagsLength());
+              List<Tag> tags = CellUtil.getTags(cell);
               if (modifiedTagFound) {
                 // Rewrite the tags by removing the modified tags.
                 removeReplicationVisibilityTag(tags);
               }
               tags.addAll(visibilityTags);
-              Cell updatedCell = new TagRewriteCell(cell, Tag.fromList(tags));
+              Cell updatedCell = new TagRewriteCell(cell, TagUtil.fromList(tags));
               updatedCells.add(updatedCell);
             }
             m.getFamilyCellMap().clear();
@@ -472,28 +471,22 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
       // cell visiblilty tags
       // have been modified
       Tag modifiedTag = null;
-      if (cell.getTagsLength() > 0) {
-        Iterator<Tag> tagsIterator = CellUtil.tagsIterator(cell.getTagsArray(),
-            cell.getTagsOffset(), cell.getTagsLength());
-        while (tagsIterator.hasNext()) {
-          Tag tag = tagsIterator.next();
-          if (tag.getType() == TagType.STRING_VIS_TAG_TYPE) {
-            modifiedTag = tag;
-            break;
-          }
+      Iterator<Tag> tagsIterator = CellUtil.tagsIterator(cell);
+      while (tagsIterator.hasNext()) {
+        Tag tag = tagsIterator.next();
+        if (tag.getType() == TagType.STRING_VIS_TAG_TYPE) {
+          modifiedTag = tag;
+          break;
         }
       }
       pair.setFirst(true);
       pair.setSecond(modifiedTag);
       return pair;
     }
-    if (cell.getTagsLength() > 0) {
-      Iterator<Tag> tagsItr = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(),
-          cell.getTagsLength());
-      while (tagsItr.hasNext()) {
-        if (RESERVED_VIS_TAG_TYPES.contains(tagsItr.next().getType())) {
-          return pair;
-        }
+    Iterator<Tag> tagsItr = CellUtil.tagsIterator(cell);
+    while (tagsItr.hasNext()) {
+      if (RESERVED_VIS_TAG_TYPES.contains(tagsItr.next().getType())) {
+        return pair;
       }
     }
     pair.setFirst(true);
@@ -520,13 +513,10 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
     if (isSystemOrSuperUser()) {
       return true;
     }
-    if (cell.getTagsLength() > 0) {
-      Iterator<Tag> tagsItr = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(),
-          cell.getTagsLength());
-      while (tagsItr.hasNext()) {
-        if (RESERVED_VIS_TAG_TYPES.contains(tagsItr.next().getType())) {
-          return false;
-        }
+    Iterator<Tag> tagsItr = CellUtil.tagsIterator(cell);
+    while (tagsItr.hasNext()) {
+      if (RESERVED_VIS_TAG_TYPES.contains(tagsItr.next().getType())) {
+        return false;
       }
     }
     return true;
@@ -739,21 +729,17 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
     boolean authCheck = authorizationEnabled && checkAuths && !(isSystemOrSuperUser());
     tags.addAll(this.visibilityLabelService.createVisibilityExpTags(cellVisibility.getExpression(),
         true, authCheck));
-    // Save an object allocation where we can
-    if (newCell.getTagsLength() > 0) {
-      // Carry forward all other tags
-      Iterator<Tag> tagsItr = CellUtil.tagsIterator(newCell.getTagsArray(),
-          newCell.getTagsOffset(), newCell.getTagsLength());
-      while (tagsItr.hasNext()) {
-        Tag tag = tagsItr.next();
-        if (tag.getType() != TagType.VISIBILITY_TAG_TYPE
-            && tag.getType() != TagType.VISIBILITY_EXP_SERIALIZATION_FORMAT_TAG_TYPE) {
-          tags.add(tag);
-        }
+    // Carry forward all other tags
+    Iterator<Tag> tagsItr = CellUtil.tagsIterator(newCell);
+    while (tagsItr.hasNext()) {
+      Tag tag = tagsItr.next();
+      if (tag.getType() != TagType.VISIBILITY_TAG_TYPE
+          && tag.getType() != TagType.VISIBILITY_EXP_SERIALIZATION_FORMAT_TAG_TYPE) {
+        tags.add(tag);
       }
     }
 
-    Cell rewriteCell = new TagRewriteCell(newCell, Tag.fromList(tags));
+    Cell rewriteCell = new TagRewriteCell(newCell, TagUtil.fromList(tags));
     return rewriteCell;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityReplicationEndpoint.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityReplicationEndpoint.java
index aca4994..3db54c6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityReplicationEndpoint.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityReplicationEndpoint.java
@@ -24,10 +24,12 @@ import java.util.UUID;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.TagRewriteCell;
 import org.apache.hadoop.hbase.TagType;
+import org.apache.hadoop.hbase.TagUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
@@ -79,7 +81,8 @@ public class VisibilityReplicationEndpoint implements ReplicationEndpoint {
                 byte[] modifiedVisExpression = visibilityLabelsService
                     .encodeVisibilityForReplication(visTags, serializationFormat);
                 if (modifiedVisExpression != null) {
-                  nonVisTags.add(new Tag(TagType.STRING_VIS_TAG_TYPE, modifiedVisExpression));
+                  nonVisTags
+                      .add(new ArrayBackedTag(TagType.STRING_VIS_TAG_TYPE, modifiedVisExpression));
                 }
               } catch (Exception ioe) {
                 LOG.error(
@@ -92,7 +95,7 @@ public class VisibilityReplicationEndpoint implements ReplicationEndpoint {
                 continue;
               }
               // Recreate the cell with the new tags and the existing tags
-              Cell newCell = new TagRewriteCell(cell, Tag.fromList(nonVisTags));
+              Cell newCell = new TagRewriteCell(cell, TagUtil.fromList(nonVisTags));
               newEdit.add(newCell);
             } else {
               newEdit.add(cell);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityUtils.java
index c725b11..1db506d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityUtils.java
@@ -35,11 +35,13 @@ import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.TagType;
+import org.apache.hadoop.hbase.TagUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.filter.Filter;
@@ -74,7 +76,7 @@ public class VisibilityUtils {
   public static final String VISIBILITY_LABEL_GENERATOR_CLASS =
       "hbase.regionserver.scan.visibility.label.generator.class";
   public static final String SYSTEM_LABEL = "system";
-  public static final Tag SORTED_ORDINAL_SERIALIZATION_FORMAT_TAG = new Tag(
+  public static final Tag SORTED_ORDINAL_SERIALIZATION_FORMAT_TAG = new ArrayBackedTag(
       TagType.VISIBILITY_EXP_SERIALIZATION_FORMAT_TAG_TYPE,
       VisibilityConstants.SORTED_ORDINAL_SERIALIZATION_FORMAT_TAG_VAL);
   private static final String COMMA = ",";
@@ -209,16 +211,13 @@ public class VisibilityUtils {
    */
   public static Byte extractVisibilityTags(Cell cell, List<Tag> tags) {
     Byte serializationFormat = null;
-    if (cell.getTagsLength() > 0) {
-      Iterator<Tag> tagsIterator = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(),
-          cell.getTagsLength());
-      while (tagsIterator.hasNext()) {
-        Tag tag = tagsIterator.next();
-        if (tag.getType() == TagType.VISIBILITY_EXP_SERIALIZATION_FORMAT_TAG_TYPE) {
-          serializationFormat = tag.getBuffer()[tag.getTagOffset()];
-        } else if (tag.getType() == VISIBILITY_TAG_TYPE) {
-          tags.add(tag);
-        }
+    Iterator<Tag> tagsIterator = CellUtil.tagsIterator(cell);
+    while (tagsIterator.hasNext()) {
+      Tag tag = tagsIterator.next();
+      if (tag.getType() == TagType.VISIBILITY_EXP_SERIALIZATION_FORMAT_TAG_TYPE) {
+        serializationFormat = TagUtil.getValueAsByte(tag);
+      } else if (tag.getType() == VISIBILITY_TAG_TYPE) {
+        tags.add(tag);
       }
     }
     return serializationFormat;
@@ -239,30 +238,23 @@ public class VisibilityUtils {
   public static Byte extractAndPartitionTags(Cell cell, List<Tag> visTags,
       List<Tag> nonVisTags) {
     Byte serializationFormat = null;
-    if (cell.getTagsLength() > 0) {
-      Iterator<Tag> tagsIterator = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(),
-          cell.getTagsLength());
-      while (tagsIterator.hasNext()) {
-        Tag tag = tagsIterator.next();
-        if (tag.getType() == TagType.VISIBILITY_EXP_SERIALIZATION_FORMAT_TAG_TYPE) {
-          serializationFormat = tag.getBuffer()[tag.getTagOffset()];
-        } else if (tag.getType() == VISIBILITY_TAG_TYPE) {
-          visTags.add(tag);
-        } else {
-          // ignore string encoded visibility expressions, will be added in replication handling
-          nonVisTags.add(tag);
-        }
+    Iterator<Tag> tagsIterator = CellUtil.tagsIterator(cell);
+    while (tagsIterator.hasNext()) {
+      Tag tag = tagsIterator.next();
+      if (tag.getType() == TagType.VISIBILITY_EXP_SERIALIZATION_FORMAT_TAG_TYPE) {
+        serializationFormat = TagUtil.getValueAsByte(tag);
+      } else if (tag.getType() == VISIBILITY_TAG_TYPE) {
+        visTags.add(tag);
+      } else {
+        // ignore string encoded visibility expressions, will be added in replication handling
+        nonVisTags.add(tag);
       }
     }
     return serializationFormat;
   }
 
   public static boolean isVisibilityTagsPresent(Cell cell) {
-    if (cell.getTagsLength() == 0) {
-      return false;
-    }
-    Iterator<Tag> tagsIterator = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(),
-        cell.getTagsLength());
+    Iterator<Tag> tagsIterator = CellUtil.tagsIterator(cell);
     while (tagsIterator.hasNext()) {
       Tag tag = tagsIterator.next();
       if (tag.getType() == VISIBILITY_TAG_TYPE) {
@@ -322,7 +314,7 @@ public class VisibilityUtils {
     if (node.isSingleNode()) {
       getLabelOrdinals(node, labelOrdinals, auths, checkAuths, ordinalProvider);
       writeLabelOrdinalsToStream(labelOrdinals, dos);
-      tags.add(new Tag(VISIBILITY_TAG_TYPE, baos.toByteArray()));
+      tags.add(new ArrayBackedTag(VISIBILITY_TAG_TYPE, baos.toByteArray()));
       baos.reset();
     } else {
       NonLeafExpressionNode nlNode = (NonLeafExpressionNode) node;
@@ -330,14 +322,14 @@ public class VisibilityUtils {
         for (ExpressionNode child : nlNode.getChildExps()) {
           getLabelOrdinals(child, labelOrdinals, auths, checkAuths, ordinalProvider);
           writeLabelOrdinalsToStream(labelOrdinals, dos);
-          tags.add(new Tag(VISIBILITY_TAG_TYPE, baos.toByteArray()));
+          tags.add(new ArrayBackedTag(VISIBILITY_TAG_TYPE, baos.toByteArray()));
           baos.reset();
           labelOrdinals.clear();
         }
       } else {
         getLabelOrdinals(nlNode, labelOrdinals, auths, checkAuths, ordinalProvider);
         writeLabelOrdinalsToStream(labelOrdinals, dos);
-        tags.add(new Tag(VISIBILITY_TAG_TYPE, baos.toByteArray()));
+        tags.add(new ArrayBackedTag(VISIBILITY_TAG_TYPE, baos.toByteArray()));
         baos.reset();
       }
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALPrettyPrinter.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALPrettyPrinter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALPrettyPrinter.java
index 5df7394..b212fe6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALPrettyPrinter.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALPrettyPrinter.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.TagUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader;
@@ -339,12 +340,10 @@ public class WALPrettyPrinter {
     stringMap.put("vlen", cell.getValueLength());
     if (cell.getTagsLength() > 0) {
       List<String> tagsString = new ArrayList<String>();
-      Iterator<Tag> tagsIterator = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(),
-          cell.getTagsLength());
+      Iterator<Tag> tagsIterator = CellUtil.tagsIterator(cell);
       while (tagsIterator.hasNext()) {
         Tag tag = tagsIterator.next();
-        tagsString.add((tag.getType()) + ":"
-            + Bytes.toStringBinary(tag.getBuffer(), tag.getTagOffset(), tag.getTagLength()));
+        tagsString.add((tag.getType()) + ":" + Bytes.toStringBinary(TagUtil.cloneValue(tag)));
       }
       stringMap.put("tag", tagsString);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
index 30629a3..821b995 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java
@@ -1405,7 +1405,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
           byte[] tag = generateData(this.rand, TAG_LENGTH);
           Tag[] tags = new Tag[opts.noOfTags];
           for (int n = 0; n < opts.noOfTags; n++) {
-            Tag t = new Tag((byte) n, tag);
+            Tag t = new ArrayBackedTag((byte) n, tag);
             tags[n] = t;
           }
           KeyValue kv = new KeyValue(row, FAMILY_NAME, qualifier, HConstants.LATEST_TIMESTAMP,
@@ -1493,7 +1493,7 @@ public class PerformanceEvaluation extends Configured implements Tool {
           byte[] tag = generateData(this.rand, TAG_LENGTH);
           Tag[] tags = new Tag[opts.noOfTags];
           for (int n = 0; n < opts.noOfTags; n++) {
-            Tag t = new Tag((byte) n, tag);
+            Tag t = new ArrayBackedTag((byte) n, tag);
             tags[n] = t;
           }
           KeyValue kv = new KeyValue(row, FAMILY_NAME, qualifier, HConstants.LATEST_TIMESTAMP,

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResultSizeEstimation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResultSizeEstimation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResultSizeEstimation.java
index f83590a..1647e97 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResultSizeEstimation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResultSizeEstimation.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
@@ -103,11 +104,11 @@ public class TestResultSizeEstimation {
     Table table = TEST_UTIL.createTable(TABLE, FAMILIES);
     Put p = new Put(ROW1);
     p.add(new KeyValue(ROW1, FAMILY, QUALIFIER, Long.MAX_VALUE, VALUE,
-      new Tag[] { new Tag((byte)1, new byte[TAG_DATA_SIZE]) } ));
+      new Tag[] { new ArrayBackedTag((byte)1, new byte[TAG_DATA_SIZE]) } ));
     table.put(p);
     p = new Put(ROW2);
     p.add(new KeyValue(ROW2, FAMILY, QUALIFIER, Long.MAX_VALUE, VALUE,
-      new Tag[] { new Tag((byte)1, new byte[TAG_DATA_SIZE]) } ));
+      new Tag[] { new ArrayBackedTag((byte)1, new byte[TAG_DATA_SIZE]) } ));
     table.put(p);
 
     Scan s = new Scan();

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java
index ce48ca1..00969b2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestDataBlockEncoders.java
@@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.codec.prefixtree.PrefixTreeSeeker;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.hfile.HFileBlock.Writer.BufferGrabbingByteArrayOutputStream;
@@ -136,10 +137,10 @@ public class TestDataBlockEncoders {
     } else {
       byte[] metaValue1 = Bytes.toBytes("metaValue1");
       byte[] metaValue2 = Bytes.toBytes("metaValue2");
-      kvList.add(new KeyValue(row, family, qualifier, 0l, value, new Tag[] { new Tag((byte) 1,
-          metaValue1) }));
-      kvList.add(new KeyValue(row, family, qualifier, 0l, value, new Tag[] { new Tag((byte) 1,
-          metaValue2) }));
+      kvList.add(new KeyValue(row, family, qualifier, 0l, value,
+          new Tag[] { new ArrayBackedTag((byte) 1, metaValue1) }));
+      kvList.add(new KeyValue(row, family, qualifier, 0l, value,
+          new Tag[] { new ArrayBackedTag((byte) 1, metaValue2) }));
     }
     testEncodersOnDataset(kvList, includesMemstoreTS, includesTags);
   }
@@ -160,10 +161,10 @@ public class TestDataBlockEncoders {
     if (includesTags) {
       byte[] metaValue1 = Bytes.toBytes("metaValue1");
       byte[] metaValue2 = Bytes.toBytes("metaValue2");
-      kvList.add(new KeyValue(row, family, qualifier, 0l, value, new Tag[] { new Tag((byte) 1,
-          metaValue1) }));
-      kvList.add(new KeyValue(row, family, qualifier, 0l, value, new Tag[] { new Tag((byte) 1,
-          metaValue2) }));
+      kvList.add(new KeyValue(row, family, qualifier, 0l, value,
+          new Tag[] { new ArrayBackedTag((byte) 1, metaValue1) }));
+      kvList.add(new KeyValue(row, family, qualifier, 0l, value,
+          new Tag[] { new ArrayBackedTag((byte) 1, metaValue2) }));
     } else {
       kvList.add(new KeyValue(row, family, qualifier, -1l, Type.Put, value));
       kvList.add(new KeyValue(row, family, qualifier, -2l, Type.Put, value));
@@ -416,10 +417,10 @@ public class TestDataBlockEncoders {
     byte[] value0 = new byte[] { 'd' };
     byte[] value1 = new byte[] { 0x00 };
     if (includesTags) {
-      kvList.add(new KeyValue(row, family, qualifier0, 0, value0, new Tag[] { new Tag((byte) 1,
-          "value1") }));
-      kvList.add(new KeyValue(row, family, qualifier1, 0, value1, new Tag[] { new Tag((byte) 1,
-          "value1") }));
+      kvList.add(new KeyValue(row, family, qualifier0, 0, value0,
+          new Tag[] { new ArrayBackedTag((byte) 1, "value1") }));
+      kvList.add(new KeyValue(row, family, qualifier1, 0, value1,
+          new Tag[] { new ArrayBackedTag((byte) 1, "value1") }));
     } else {
       kvList.add(new KeyValue(row, family, qualifier0, 0, Type.Put, value0));
       kvList.add(new KeyValue(row, family, qualifier1, 0, Type.Put, value1));

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestEncodedSeekers.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestEncodedSeekers.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestEncodedSeekers.java
index ce66e82..0869df6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestEncodedSeekers.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestEncodedSeekers.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Put;
@@ -148,7 +149,7 @@ public class TestEncodedSeekers {
         byte[] value = dataGenerator.generateRandomSizeValue(key, col);
         if (includeTags) {
           Tag[] tag = new Tag[1];
-          tag[0] = new Tag((byte) 1, "Visibility");
+          tag[0] = new ArrayBackedTag((byte) 1, "Visibility");
           KeyValue kv = new KeyValue(key, CF_BYTES, col, HConstants.LATEST_TIMESTAMP, value, tag);
           put.add(kv);
         } else {

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestPrefixTreeEncoding.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestPrefixTreeEncoding.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestPrefixTreeEncoding.java
index 031bf25..fd9b90b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestPrefixTreeEncoding.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/encoding/TestPrefixTreeEncoding.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.codec.prefixtree.PrefixTreeCodec;
 import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoder.EncodedSeeker;
@@ -280,7 +281,7 @@ public class TestPrefixTreeEncoding {
           kvset.add(kv);
         } else {
           KeyValue kv = new KeyValue(getRowKey(batchId, i), CF_BYTES, getQualifier(j), 0l,
-              getValue(batchId, i, j), new Tag[] { new Tag((byte) 1, "metaValue1") });
+              getValue(batchId, i, j), new Tag[] { new ArrayBackedTag((byte) 1, "metaValue1") });
           kvset.add(kv);
         }
       }
@@ -308,7 +309,7 @@ public class TestPrefixTreeEncoding {
           kvset.add(kv);
         } else {
           KeyValue kv = new KeyValue(getRowKey(batchId, i), CF_BYTES, getQualifier(j), 0l,
-              getValue(batchId, i, j), new Tag[] { new Tag((byte) 1, "metaValue1") });
+              getValue(batchId, i, j), new Tag[] { new ArrayBackedTag((byte) 1, "metaValue1") });
           kvset.add(kv);
         }
       }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
index ccf59a4..5158e35 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestCacheOnWrite.java
@@ -39,6 +39,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
@@ -385,7 +386,7 @@ public class TestCacheOnWrite {
       byte[] value = RandomKeyValueUtil.randomValue(rand);
       KeyValue kv;
       if(useTags) {
-        Tag t = new Tag((byte) 1, "visibility");
+        Tag t = new ArrayBackedTag((byte) 1, "visibility");
         List<Tag> tagList = new ArrayList<Tag>();
         tagList.add(t);
         Tag[] tags = new Tag[1];
@@ -434,7 +435,7 @@ public class TestCacheOnWrite {
           String valueStr = "value_" + rowStr + "_" + qualStr;
           for (int iTS = 0; iTS < 5; ++iTS) {
             if (useTags) {
-              Tag t = new Tag((byte) 1, "visibility");
+              Tag t = new ArrayBackedTag((byte) 1, "visibility");
               Tag[] tags = new Tag[1];
               tags[0] = t;
               KeyValue kv = new KeyValue(Bytes.toBytes(rowStr), cfBytes, Bytes.toBytes(qualStr),

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
index 929ad8a..66fb49c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFile.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.hfile.HFile.Reader;
 import org.apache.hadoop.hbase.io.hfile.HFile.Writer;
@@ -169,7 +170,7 @@ public class TestHFile  {
     for (int i = start; i < (start + n); i++) {
       String key = String.format(localFormatter, Integer.valueOf(i));
       if (useTags) {
-        Tag t = new Tag((byte) 1, "myTag1");
+        Tag t = new ArrayBackedTag((byte) 1, "myTag1");
         Tag[] tags = new Tag[1];
         tags[0] = t;
         kv = new KeyValue(Bytes.toBytes(key), Bytes.toBytes("family"), Bytes.toBytes("qual"),

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java
index 12fb584..4ee7f5b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java
@@ -52,6 +52,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.fs.HFileSystem;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
@@ -167,8 +168,8 @@ public class TestHFileBlock {
       if (!useTag) {
         keyValues.add(new KeyValue(row, family, qualifier, timestamp, value));
       } else {
-        keyValues.add(new KeyValue(row, family, qualifier, timestamp, value, new Tag[] { new Tag(
-            (byte) 1, Bytes.toBytes("myTagVal")) }));
+        keyValues.add(new KeyValue(row, family, qualifier, timestamp, value,
+            new Tag[] { new ArrayBackedTag((byte) 1, Bytes.toBytes("myTagVal")) }));
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java
index 15aa912..c7eb11b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileWriterV3.java
@@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
 import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
@@ -140,7 +141,7 @@ public class TestHFileWriterV3 {
         for (int j = 0; j < 1 + rand.nextInt(4); j++) {
           byte[] tagBytes = new byte[16];
           rand.nextBytes(tagBytes);
-          tags.add(new Tag((byte) 1, tagBytes));
+          tags.add(new ArrayBackedTag((byte) 1, tagBytes));
         }
         keyValue = new KeyValue(keyBytes, null, null, HConstants.LATEST_TIMESTAMP,
             valueBytes, tags);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestReseekTo.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestReseekTo.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestReseekTo.java
index a17368c..90e398d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestReseekTo.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestReseekTo.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.testclassification.IOTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -84,7 +85,7 @@ public class TestReseekTo {
             Bytes.toBytes(value));
         writer.append(kv);
       } else if (tagUsage == TagUsage.ONLY_TAG) {
-        Tag t = new Tag((byte) 1, "myTag1");
+        Tag t = new ArrayBackedTag((byte) 1, "myTag1");
         Tag[] tags = new Tag[1];
         tags[0] = t;
         kv = new KeyValue(Bytes.toBytes(key), Bytes.toBytes("family"), Bytes.toBytes("qual"),
@@ -92,7 +93,7 @@ public class TestReseekTo {
         writer.append(kv);
       } else {
         if (key % 4 == 0) {
-          Tag t = new Tag((byte) 1, "myTag1");
+          Tag t = new ArrayBackedTag((byte) 1, "myTag1");
           Tag[] tags = new Tag[1];
           tags[0] = t;
           kv = new KeyValue(Bytes.toBytes(key), Bytes.toBytes("family"), Bytes.toBytes("qual"),

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekTo.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekTo.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekTo.java
index c1d91ec..6eead71 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekTo.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestSeekTo.java
@@ -45,6 +45,8 @@ import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.OffheapKeyValue;
 import org.apache.hadoop.hbase.ShareableMemory;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.TagUtil;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.testclassification.IOTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
@@ -88,7 +90,7 @@ public class TestSeekTo {
       return new KeyValue(Bytes.toBytes(row), Bytes.toBytes("family"), Bytes.toBytes("qualifier"),
           Bytes.toBytes("value"));
     } else if (tagUsage == TagUsage.ONLY_TAG) {
-      Tag t = new Tag((byte) 1, "myTag1");
+      Tag t = new ArrayBackedTag((byte) 1, "myTag1");
       Tag[] tags = new Tag[1];
       tags[0] = t;
       return new KeyValue(Bytes.toBytes(row), Bytes.toBytes("family"), Bytes.toBytes("qualifier"),
@@ -100,7 +102,7 @@ public class TestSeekTo {
             Bytes.toBytes("qualifier"), HConstants.LATEST_TIMESTAMP, Bytes.toBytes("value"));
       } else {
         switchKVs = false;
-        Tag t = new Tag((byte) 1, "myTag1");
+        Tag t = new ArrayBackedTag((byte) 1, "myTag1");
         Tag[] tags = new Tag[1];
         tags[0] = t;
         return new KeyValue(Bytes.toBytes(row), Bytes.toBytes("family"),
@@ -174,11 +176,10 @@ public class TestSeekTo {
     assertEquals("i", toRowStr(scanner.getCell()));
     Cell cell = scanner.getCell();
     if (tagUsage != TagUsage.NO_TAG && cell.getTagsLength() > 0) {
-      Iterator<Tag> tagsIterator = CellUtil.tagsIterator(cell.getTagsArray(), cell.getTagsOffset(),
-          cell.getTagsLength());
+      Iterator<Tag> tagsIterator = CellUtil.tagsIterator(cell);
       while (tagsIterator.hasNext()) {
         Tag next = tagsIterator.next();
-        assertEquals("myTag1", Bytes.toString(next.getValue()));
+        assertEquals("myTag1", Bytes.toString(TagUtil.cloneValue(next)));
       }
     }
     assertTrue(scanner.seekBefore(toKV("k", tagUsage)));

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
index 47b6b5c..ef02431 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHMobStore.java
@@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.TagType;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Scan;
@@ -57,7 +58,6 @@ import org.apache.hadoop.hbase.io.hfile.HFile;
 import org.apache.hadoop.hbase.mob.MobConstants;
 import org.apache.hadoop.hbase.mob.MobUtils;
 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
-import org.apache.hadoop.hbase.regionserver.StoreFile.Reader;
 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
 import org.apache.hadoop.hbase.regionserver.compactions.NoLimitCompactionThroughputController;
 import org.apache.hadoop.hbase.security.EncryptionUtil;
@@ -190,7 +190,8 @@ public class TestHMobStore {
 
     String targetPathName = MobUtils.formatDate(currentDate);
     byte[] referenceValue = Bytes.toBytes(targetPathName + Path.SEPARATOR + mobFilePath.getName());
-    Tag tableNameTag = new Tag(TagType.MOB_TABLE_NAME_TAG_TYPE, store.getTableName().getName());
+    Tag tableNameTag = new ArrayBackedTag(TagType.MOB_TABLE_NAME_TAG_TYPE,
+        store.getTableName().getName());
     KeyValue kv1 = new KeyValue(row, family, qf1, Long.MAX_VALUE, referenceValue);
     KeyValue kv2 = new KeyValue(row, family, qf2, Long.MAX_VALUE, referenceValue);
     KeyValue kv3 = new KeyValue(row2, family, qf3, Long.MAX_VALUE, referenceValue);

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
index 4582e31..cef92a6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
@@ -96,7 +96,7 @@ import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.RegionTooBusyException;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.TagType;
 import org.apache.hadoop.hbase.Waiter;
 import org.apache.hadoop.hbase.client.Append;
@@ -6335,16 +6335,16 @@ public class TestHRegion {
       long now = EnvironmentEdgeManager.currentTime();
       // Add a cell that will expire in 5 seconds via cell TTL
       region.put(new Put(row).add(new KeyValue(row, fam1, q1, now,
-        HConstants.EMPTY_BYTE_ARRAY, new Tag[] {
+        HConstants.EMPTY_BYTE_ARRAY, new ArrayBackedTag[] {
           // TTL tags specify ts in milliseconds
-          new Tag(TagType.TTL_TAG_TYPE, Bytes.toBytes(5000L)) } )));
+          new ArrayBackedTag(TagType.TTL_TAG_TYPE, Bytes.toBytes(5000L)) } )));
       // Add a cell that will expire after 10 seconds via family setting
       region.put(new Put(row).addColumn(fam1, q2, now, HConstants.EMPTY_BYTE_ARRAY));
       // Add a cell that will expire in 15 seconds via cell TTL
       region.put(new Put(row).add(new KeyValue(row, fam1, q3, now + 10000 - 1,
-        HConstants.EMPTY_BYTE_ARRAY, new Tag[] {
+        HConstants.EMPTY_BYTE_ARRAY, new ArrayBackedTag[] {
           // TTL tags specify ts in milliseconds
-          new Tag(TagType.TTL_TAG_TYPE, Bytes.toBytes(5000L)) } )));
+          new ArrayBackedTag(TagType.TTL_TAG_TYPE, Bytes.toBytes(5000L)) } )));
       // Add a cell that will expire in 20 seconds via family setting
       region.put(new Put(row).addColumn(fam1, q4, now + 10000 - 1, HConstants.EMPTY_BYTE_ARRAY));
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileScannerWithTagCompression.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileScannerWithTagCompression.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileScannerWithTagCompression.java
index 1bcb7c9..3c062f8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileScannerWithTagCompression.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFileScannerWithTagCompression.java
@@ -31,9 +31,11 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
+import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.TagUtil;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
 import org.apache.hadoop.hbase.io.hfile.HFileContext;
@@ -86,7 +88,7 @@ public class TestStoreFileScannerWithTagCompression {
           kv.getRowLength()));
       List<Tag> tags = KeyValueUtil.ensureKeyValue(kv).getTags();
       assertEquals(1, tags.size());
-      assertEquals("tag3", Bytes.toString(tags.get(0).getValue()));
+      assertEquals("tag3", Bytes.toString(TagUtil.cloneValue(tags.get(0))));
     } finally {
       s.close();
     }
@@ -97,9 +99,9 @@ public class TestStoreFileScannerWithTagCompression {
     byte[] qualifier = Bytes.toBytes("q");
     long now = System.currentTimeMillis();
     byte[] b = Bytes.toBytes("k1");
-    Tag t1 = new Tag((byte) 1, "tag1");
-    Tag t2 = new Tag((byte) 2, "tag2");
-    Tag t3 = new Tag((byte) 3, "tag3");
+    Tag t1 = new ArrayBackedTag((byte) 1, "tag1");
+    Tag t2 = new ArrayBackedTag((byte) 2, "tag2");
+    Tag t3 = new ArrayBackedTag((byte) 3, "tag3");
     try {
       writer.append(new KeyValue(b, fam, qualifier, now, b, new Tag[] { t1 }));
       b = Bytes.toBytes("k3");

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTags.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTags.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTags.java
index a85e479..0f7f23a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTags.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestTags.java
@@ -36,6 +36,8 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.TagUtil;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Append;
 import org.apache.hadoop.hbase.client.Durability;
@@ -325,7 +327,7 @@ public class TestTags {
             if (CellUtil.matchingRow(current, row)) {
               assertEquals(1, TestCoprocessorForTags.tags.size());
               Tag tag = TestCoprocessorForTags.tags.get(0);
-              assertEquals(bigTagLen, tag.getTagLength());
+              assertEquals(bigTagLen, tag.getValueLength());
             } else {
               assertEquals(0, TestCoprocessorForTags.tags.size());
             }
@@ -350,7 +352,7 @@ public class TestTags {
             if (CellUtil.matchingRow(current, row)) {
               assertEquals(1, TestCoprocessorForTags.tags.size());
               Tag tag = TestCoprocessorForTags.tags.get(0);
-              assertEquals(bigTagLen, tag.getTagLength());
+              assertEquals(bigTagLen, tag.getValueLength());
             } else {
               assertEquals(0, TestCoprocessorForTags.tags.size());
             }
@@ -403,7 +405,7 @@ public class TestTags {
       List<Tag> tags = TestCoprocessorForTags.tags;
       assertEquals(3L, Bytes.toLong(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength()));
       assertEquals(1, tags.size());
-      assertEquals("tag1", Bytes.toString(tags.get(0).getValue()));
+      assertEquals("tag1", Bytes.toString(TagUtil.cloneValue(tags.get(0))));
       TestCoprocessorForTags.checkTagPresence = false;
       TestCoprocessorForTags.tags = null;
 
@@ -421,7 +423,7 @@ public class TestTags {
       // We cannot assume the ordering of tags
       List<String> tagValues = new ArrayList<String>();
       for (Tag tag: tags) {
-        tagValues.add(Bytes.toString(tag.getValue()));
+        tagValues.add(Bytes.toString(TagUtil.cloneValue(tag)));
       }
       assertTrue(tagValues.contains("tag1"));
       assertTrue(tagValues.contains("tag2"));
@@ -445,7 +447,7 @@ public class TestTags {
       tags = TestCoprocessorForTags.tags;
       assertEquals(4L, Bytes.toLong(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength()));
       assertEquals(1, tags.size());
-      assertEquals("tag2", Bytes.toString(tags.get(0).getValue()));
+      assertEquals("tag2", Bytes.toString(TagUtil.cloneValue(tags.get(0))));
       TestCoprocessorForTags.checkTagPresence = false;
       TestCoprocessorForTags.tags = null;
 
@@ -466,7 +468,7 @@ public class TestTags {
       kv = KeyValueUtil.ensureKeyValue(result.getColumnLatestCell(f, q));
       tags = TestCoprocessorForTags.tags;
       assertEquals(1, tags.size());
-      assertEquals("tag1", Bytes.toString(tags.get(0).getValue()));
+      assertEquals("tag1", Bytes.toString(TagUtil.cloneValue(tags.get(0))));
       TestCoprocessorForTags.checkTagPresence = false;
       TestCoprocessorForTags.tags = null;
 
@@ -483,7 +485,7 @@ public class TestTags {
       // We cannot assume the ordering of tags
       tagValues.clear();
       for (Tag tag: tags) {
-        tagValues.add(Bytes.toString(tag.getValue()));
+        tagValues.add(Bytes.toString(TagUtil.cloneValue(tag)));
       }
       assertTrue(tagValues.contains("tag1"));
       assertTrue(tagValues.contains("tag2"));
@@ -506,7 +508,7 @@ public class TestTags {
       kv = KeyValueUtil.ensureKeyValue(result.getColumnLatestCell(f, q));
       tags = TestCoprocessorForTags.tags;
       assertEquals(1, tags.size());
-      assertEquals("tag2", Bytes.toString(tags.get(0).getValue()));
+      assertEquals("tag2", Bytes.toString(TagUtil.cloneValue(tags.get(0))));
     } finally {
       TestCoprocessorForTags.checkTagPresence = false;
       TestCoprocessorForTags.tags = null;
@@ -569,7 +571,7 @@ public class TestTags {
             if (cf == null) {
               cf = CellUtil.cloneFamily(kv);
             }
-            Tag tag = new Tag((byte) 1, attribute);
+            Tag tag = new ArrayBackedTag((byte) 1, attribute);
             List<Tag> tagList = new ArrayList<Tag>();
             tagList.add(tag);
 
@@ -611,7 +613,7 @@ public class TestTags {
           CellScanner cellScanner = result.cellScanner();
           if (cellScanner.advance()) {
             Cell cell = cellScanner.current();
-            tags = Tag.asList(cell.getTagsArray(), cell.getTagsOffset(),
+            tags = TagUtil.asList(cell.getTagsArray(), cell.getTagsOffset(),
                 cell.getTagsLength());
           }
         }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestKeyValueCompression.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestKeyValueCompression.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestKeyValueCompression.java
index 0450904..104f897 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestKeyValueCompression.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestKeyValueCompression.java
@@ -24,9 +24,10 @@ import java.util.List;
 
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.testclassification.RegionServerTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.io.util.LRUDictionary;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.DataOutputBuffer;
@@ -108,7 +109,7 @@ public class TestKeyValueCompression {
     byte[] value = Bytes.toBytes("myValue");
     List<Tag> tags = new ArrayList<Tag>(noOfTags);
     for (int i = 1; i <= noOfTags; i++) {
-      tags.add(new Tag((byte) i, Bytes.toBytes("tagValue" + i)));
+      tags.add(new ArrayBackedTag((byte) i, Bytes.toBytes("tagValue" + i)));
     }
     return new KeyValue(row, cf, q, HConstants.LATEST_TIMESTAMP, value, tags);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALCellCodecWithCompression.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALCellCodecWithCompression.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALCellCodecWithCompression.java
index 501fdda..e834ac8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALCellCodecWithCompression.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestWALCellCodecWithCompression.java
@@ -30,6 +30,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.TagUtil;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.codec.Codec.Decoder;
 import org.apache.hadoop.hbase.codec.Codec.Encoder;
 import org.apache.hadoop.hbase.io.util.LRUDictionary;
@@ -69,7 +71,7 @@ public class TestWALCellCodecWithCompression {
     KeyValue kv = (KeyValue) decoder.current();
     List<Tag> tags = kv.getTags();
     assertEquals(1, tags.size());
-    assertEquals("tagValue1", Bytes.toString(tags.get(0).getValue()));
+    assertEquals("tagValue1", Bytes.toString(TagUtil.cloneValue(tags.get(0))));
     decoder.advance();
     kv = (KeyValue) decoder.current();
     tags = kv.getTags();
@@ -78,8 +80,8 @@ public class TestWALCellCodecWithCompression {
     kv = (KeyValue) decoder.current();
     tags = kv.getTags();
     assertEquals(2, tags.size());
-    assertEquals("tagValue1", Bytes.toString(tags.get(0).getValue()));
-    assertEquals("tagValue2", Bytes.toString(tags.get(1).getValue()));
+    assertEquals("tagValue1", Bytes.toString(TagUtil.cloneValue(tags.get(0))));
+    assertEquals("tagValue2", Bytes.toString(TagUtil.cloneValue(tags.get(1))));
   }
 
   private KeyValue createKV(int noOfTags) {
@@ -89,7 +91,7 @@ public class TestWALCellCodecWithCompression {
     byte[] value = Bytes.toBytes("myValue");
     List<Tag> tags = new ArrayList<Tag>(noOfTags);
     for (int i = 1; i <= noOfTags; i++) {
-      tags.add(new Tag((byte) i, Bytes.toBytes("tagValue" + i)));
+      tags.add(new ArrayBackedTag((byte) i, Bytes.toBytes("tagValue" + i)));
     }
     return new KeyValue(row, cf, q, HConstants.LATEST_TIMESTAMP, value, tags);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java
index 988373f..8bfdc2a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWithTags.java
@@ -39,6 +39,8 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.TagUtil;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
@@ -209,7 +211,7 @@ public class TestReplicationWithTags {
             if (cf == null) {
               cf = CellUtil.cloneFamily(kv);
             }
-            Tag tag = new Tag(TAG_TYPE, attribute);
+            Tag tag = new ArrayBackedTag(TAG_TYPE, attribute);
             List<Tag> tagList = new ArrayList<Tag>();
             tagList.add(tag);
 
@@ -238,7 +240,7 @@ public class TestReplicationWithTags {
         // Check tag presence in the 1st cell in 1st Result
         if (!results.isEmpty()) {
           Cell cell = results.get(0);
-          tags = Tag.asList(cell.getTagsArray(), cell.getTagsOffset(), cell.getTagsLength());
+          tags = TagUtil.asList(cell.getTagsArray(), cell.getTagsOffset(), cell.getTagsLength());
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/a9b671b3/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
index 8ecc6e3..9f20c11 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
@@ -56,6 +56,7 @@ import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.TableNotFoundException;
 import org.apache.hadoop.hbase.Tag;
+import org.apache.hadoop.hbase.ArrayBackedTag;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Append;
 import org.apache.hadoop.hbase.client.Connection;
@@ -2516,7 +2517,7 @@ public class TestAccessController extends SecureTestUtil {
             Table t = conn.getTable(TEST_TABLE);) {
           KeyValue kv = new KeyValue(TEST_ROW, TEST_FAMILY, TEST_QUALIFIER,
             HConstants.LATEST_TIMESTAMP, HConstants.EMPTY_BYTE_ARRAY,
-            new Tag[] { new Tag(AccessControlLists.ACL_TAG_TYPE,
+            new Tag[] { new ArrayBackedTag(AccessControlLists.ACL_TAG_TYPE,
               ProtobufUtil.toUsersAndPermissions(USER_OWNER.getShortName(),
                 new Permission(Permission.Action.READ)).toByteArray()) });
           t.put(new Put(TEST_ROW).add(kv));