You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ra...@apache.org on 2015/04/29 20:16:46 UTC

phoenix git commit: PHOENIX-1856 Include min row key for each region in stats row (ram)

Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-1.0 d5b01db97 -> 5da66200c


PHOENIX-1856 Include min row key for each region in stats row (ram)


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

Branch: refs/heads/4.x-HBase-1.0
Commit: 5da66200ce3a30607e482380e6e498d7cd00f75c
Parents: d5b01db
Author: ramkrishna <ra...@gmail.com>
Authored: Wed Apr 29 23:46:05 2015 +0530
Committer: ramkrishna <ra...@gmail.com>
Committed: Wed Apr 29 23:46:05 2015 +0530

----------------------------------------------------------------------
 .../StatsCollectorWithSplitsAndMultiCFIT.java   | 32 ++++++++++-----
 .../org/apache/phoenix/schema/PTableImpl.java   |  4 +-
 .../phoenix/schema/stats/GuidePostsInfo.java    |  9 +++--
 .../schema/stats/StatisticsCollector.java       | 31 ++++++++++++++-
 .../phoenix/schema/stats/StatisticsUtil.java    |  2 +
 .../phoenix/schema/stats/StatisticsWriter.java  | 41 +++++++++++++++-----
 6 files changed, 95 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/5da66200/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorWithSplitsAndMultiCFIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorWithSplitsAndMultiCFIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorWithSplitsAndMultiCFIT.java
index bcb3a0a..88a8215 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorWithSplitsAndMultiCFIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorWithSplitsAndMultiCFIT.java
@@ -105,9 +105,12 @@ public class StatsCollectorWithSplitsAndMultiCFIT extends StatsCollectorAbstract
         List<HRegionLocation> regions = services.getAllTableRegions(STATS_TEST_TABLE_NEW_BYTES);
         assertEquals(1, regions.size());
 
-        rs = conn.createStatement().executeQuery(
-                "SELECT GUIDE_POSTS_COUNT, REGION_NAME, GUIDE_POSTS_ROW_COUNT FROM SYSTEM.STATS WHERE PHYSICAL_NAME='"
-                        + STATS_TEST_TABLE_NAME_NEW + "' AND REGION_NAME IS NOT NULL");
+        rs =
+                conn.createStatement()
+                        .executeQuery(
+                            "SELECT GUIDE_POSTS_COUNT, REGION_NAME, GUIDE_POSTS_ROW_COUNT, " +
+                            "MIN_KEY FROM SYSTEM.STATS WHERE PHYSICAL_NAME='"
+                                    + STATS_TEST_TABLE_NAME_NEW + "' AND REGION_NAME IS NOT NULL");
         assertTrue(rs.next());
         assertEquals(11, (rs.getLong(1)));
         assertEquals(regions.get(0).getRegionInfo().getRegionNameAsString(), rs.getString(2));
@@ -187,13 +190,15 @@ public class StatsCollectorWithSplitsAndMultiCFIT extends StatsCollectorAbstract
         assertEquals(1, regions.size());
 
         rs = conn.createStatement().executeQuery(
-                "SELECT GUIDE_POSTS_COUNT, REGION_NAME, GUIDE_POSTS_ROW_COUNT FROM SYSTEM.STATS WHERE PHYSICAL_NAME='"
+                "SELECT GUIDE_POSTS_COUNT, REGION_NAME, GUIDE_POSTS_ROW_COUNT, MIN_KEY " +
+                " FROM SYSTEM.STATS WHERE PHYSICAL_NAME='"
                         + STATS_TEST_TABLE_NAME + "' AND REGION_NAME IS NOT NULL");
         assertTrue(rs.next());
         assertEquals(nRows / 2, (rs.getLong(1)));
         assertEquals(regions.get(0).getRegionInfo().getRegionNameAsString(), rs.getString(2));
         //PhoenixArray arr = (PhoenixArray)rs.getArray(3);
         assertEquals(20, rs.getLong(3));
+        assertTrue(Bytes.toString(rs.getBytes(4)).startsWith("a"));
         assertFalse(rs.next());
 
         byte[] midPoint = Bytes.toBytes(Character.toString((char)('a' + (5))));
@@ -204,19 +209,24 @@ public class StatsCollectorWithSplitsAndMultiCFIT extends StatsCollectorAbstract
 
         regions = services.getAllTableRegions(STATS_TEST_TABLE_BYTES);
         assertEquals(2, regions.size());
-        rs = conn.createStatement().executeQuery(
-                "SELECT GUIDE_POSTS_COUNT, REGION_NAME, "
-                        + "GUIDE_POSTS_ROW_COUNT FROM SYSTEM.STATS WHERE PHYSICAL_NAME='" + STATS_TEST_TABLE_NAME
-                        + "' AND REGION_NAME IS NOT NULL");
+        rs =
+                conn.createStatement()
+                        .executeQuery(
+                            "SELECT GUIDE_POSTS_COUNT, REGION_NAME, "
+                                    + "GUIDE_POSTS_ROW_COUNT, MIN_KEY " +
+                                    " FROM SYSTEM.STATS WHERE PHYSICAL_NAME='"
+                                    + STATS_TEST_TABLE_NAME + "' AND REGION_NAME IS NOT NULL");
         assertTrue(rs.next());
         assertEquals(2, rs.getLong(1));
         assertEquals(regions.get(0).getRegionInfo().getRegionNameAsString(), rs.getString(2));
         //assertEquals(5, rs.getLong(3));
+        assertTrue(Bytes.toString(rs.getBytes(4)).startsWith("a"));
         assertTrue(rs.next());
         assertEquals(8, rs.getLong(1));
         assertEquals(regions.get(1).getRegionInfo().getRegionNameAsString(), rs.getString(2));
         // This could even be 15 if the compaction thread completes after the update from split
         //assertEquals(16, rs.getLong(3));
+        assertTrue(Bytes.toString(rs.getBytes(4)).startsWith("f"));
         assertFalse(rs.next());
 
         byte[] midPoint2 = Bytes.toBytes("cj");
@@ -228,20 +238,24 @@ public class StatsCollectorWithSplitsAndMultiCFIT extends StatsCollectorAbstract
         regions = services.getAllTableRegions(STATS_TEST_TABLE_BYTES);
         assertEquals(3, regions.size());
         rs = conn.createStatement().executeQuery(
-                "SELECT GUIDE_POSTS_COUNT, REGION_NAME, GUIDE_POSTS_ROW_COUNT FROM SYSTEM.STATS WHERE PHYSICAL_NAME='"
+                "SELECT GUIDE_POSTS_COUNT, REGION_NAME, GUIDE_POSTS_ROW_COUNT, MIN_KEY " +
+                "FROM SYSTEM.STATS WHERE PHYSICAL_NAME='"
                         + STATS_TEST_TABLE_NAME + "' AND REGION_NAME IS NOT NULL");
         assertTrue(rs.next());
         assertEquals(1, rs.getLong(1));
         assertEquals(regions.get(0).getRegionInfo().getRegionNameAsString(), rs.getString(2));
         // This value varies based on whether compaction updates or split updates the GPs
         //assertEquals(3, rs.getLong(3));
+        assertTrue(Bytes.toString(rs.getBytes(4)).startsWith("a"));
         assertTrue(rs.next());
         assertEquals(1, rs.getLong(1));
         assertEquals(regions.get(1).getRegionInfo().getRegionNameAsString(), rs.getString(2));
+        assertTrue(Bytes.toString(rs.getBytes(4)).startsWith("d"));
         //assertEquals(2, rs.getLong(3));
         assertTrue(rs.next());
         assertEquals(8, rs.getLong(1));
         assertEquals(regions.get(2).getRegionInfo().getRegionNameAsString(), rs.getString(2));
+        assertTrue(Bytes.toString(rs.getBytes(4)).startsWith("f"));
         //assertEquals(16, rs.getLong(3));
         assertFalse(rs.next());
         rs = conn.createStatement().executeQuery("EXPLAIN SELECT * FROM " + STATS_TEST_TABLE_NAME);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5da66200/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
index 088595b..2d523ff 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
@@ -935,7 +935,9 @@ public class PTableImpl implements PTable {
             }
             long guidePostsByteCount = pGuidePosts.getByteCount();
             long rowCount = pGuidePosts.getRowCount();
-            GuidePostsInfo info = new GuidePostsInfo(guidePostsByteCount, value, rowCount);
+            // TODO : Not exposing MIN/MAX key outside to client 
+            GuidePostsInfo info =
+                    new GuidePostsInfo(guidePostsByteCount, value, rowCount);
             tableGuidePosts.put(pTableStatsProto.getKey().toByteArray(), info);
       }
       PTableStats stats = new PTableStatsImpl(tableGuidePosts, table.getStatsTimeStamp());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5da66200/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/GuidePostsInfo.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/GuidePostsInfo.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/GuidePostsInfo.java
index cd5d37d..b8b64ae 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/GuidePostsInfo.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/GuidePostsInfo.java
@@ -85,7 +85,7 @@ public class GuidePostsInfo {
     public void incrementRowCount() {
         this.rowCount++;
     }
-    
+
     /**
      * Combines the GuidePosts per region into one.
      * @param oldInfo
@@ -124,7 +124,6 @@ public class GuidePostsInfo {
      * reached while collecting stats.
      * @param row
      * @param byteCount
-     * @param rowCount
      * @return
      */
     public boolean addGuidePost(byte[] row, long byteCount) {
@@ -145,9 +144,11 @@ public class GuidePostsInfo {
      * @param buf
      * @param offset
      * @param l
-     * @return
+     * @param rowCount
+     * @return the GuidePostsInfo instance formed by deserializing the byte[]
      */
-    public static GuidePostsInfo deserializeGuidePostsInfo(byte[] buf, int offset, int l, long rowCount) {
+    public static GuidePostsInfo deserializeGuidePostsInfo(byte[] buf, int offset, int l,
+            long rowCount) {
         try {
             ByteArrayInputStream bytesIn = new ByteArrayInputStream(buf, offset, l);
             try {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5da66200/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsCollector.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsCollector.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsCollector.java
index d6f25c4..382cead 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsCollector.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsCollector.java
@@ -25,6 +25,7 @@ import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.client.Mutation;
@@ -66,6 +67,10 @@ public class StatisticsCollector {
     private Map<ImmutableBytesPtr, Pair<Long,GuidePostsInfo>> guidePostsMap = Maps.newHashMap();
     protected StatisticsWriter statsTable;
     private Pair<Long,GuidePostsInfo> cachedGps = null;
+    // Store minKey and maxKey at the tracker level itself so that we don't need to do 
+    // any comparison for the min/max values for every next() call
+    private byte[] minKey;
+    private int minKeyOffset, minKeyLength;
 
     public StatisticsCollector(RegionCoprocessorEnvironment env, String tableName, long clientTimeStamp) throws IOException {
         this(env, tableName, clientTimeStamp, null, null, null);
@@ -139,6 +144,8 @@ public class StatisticsCollector {
                 statsTable.addStats((region.getRegionInfo().getRegionName()), this, fam,
                         mutations);
             }
+            // Clear minKey and maxKey
+            clearMinKeys();
         } catch (IOException e) {
             logger.error("Failed to update statistics table!", e);
             throw e;
@@ -162,6 +169,12 @@ public class StatisticsCollector {
             rowTracker = 
                     new ArrayList<GuidePostsInfo>();
         }
+        if (minKey == null) {
+            Cell minCell = results.get(0);
+            minKey = minCell.getRowArray();
+            minKeyOffset =  minCell.getRowOffset();
+            minKeyLength = minCell.getRowLength();
+        }
         for (Cell cell : results) {
             KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
             maxTimeStamp = Math.max(maxTimeStamp, kv.getTimestamp());
@@ -238,9 +251,16 @@ public class StatisticsCollector {
     public void clear() {
         this.guidePostsMap.clear();
         maxTimeStamp = MetaDataProtocol.MIN_TABLE_TIMESTAMP;
+        clearMinKeys();
+    }
+
+    private void clearMinKeys() {
+        minKey = null;
+        minKeyOffset = minKeyLength = 0;
     }
 
-    public void addGuidePost(ImmutableBytesPtr cfKey, GuidePostsInfo info, long byteSize, long timestamp) {
+    public void addGuidePost(ImmutableBytesPtr cfKey, GuidePostsInfo info, long byteSize,
+            long timestamp, byte[] minKey) {
         Pair<Long, GuidePostsInfo> newInfo = new Pair<Long, GuidePostsInfo>(byteSize, info);
         Pair<Long, GuidePostsInfo> oldInfo = guidePostsMap.put(cfKey, newInfo);
         if (oldInfo != null) {
@@ -248,6 +268,9 @@ public class StatisticsCollector {
             newInfo.setFirst(oldInfo.getFirst() + newInfo.getFirst());
         }
         maxTimeStamp = Math.max(maxTimeStamp, timestamp);
+        this.minKey = minKey;
+        this.minKeyOffset = 0;
+        this.minKeyLength = this.minKey.length;
     }
 
     public GuidePostsInfo getGuidePosts(ImmutableBytesPtr fam) {
@@ -257,4 +280,10 @@ public class StatisticsCollector {
         }
         return null;
     }
+
+    public void getMinKey(ImmutableBytesWritable ptr) {
+        if (minKey != null) {
+            ptr.set(minKey, minKeyOffset, minKeyLength);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5da66200/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsUtil.java
index aab3a11..ebb7d1f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsUtil.java
@@ -80,6 +80,8 @@ public class StatisticsUtil {
         get.addColumn(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH_BYTES);
         get.addColumn(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, PhoenixDatabaseMetaData.GUIDE_POSTS_BYTES);
         get.addColumn(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, PhoenixDatabaseMetaData.GUIDE_POSTS_ROW_COUNT_BYTES);
+        get.addColumn(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES,
+            PhoenixDatabaseMetaData.MIN_KEY_BYTES);
         return statsHTable.get(get);
     }
     

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5da66200/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsWriter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsWriter.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsWriter.java
index f2d4fb5..8756568 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsWriter.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsWriter.java
@@ -24,8 +24,11 @@ import java.util.Collections;
 import java.util.List;
 
 import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.HTableInterface;
@@ -33,6 +36,7 @@ import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
@@ -84,6 +88,7 @@ public class StatisticsWriter implements Closeable {
     private final HTableInterface statsReaderTable;
     private final byte[] tableName;
     private final long clientTimeStamp;
+    private final ImmutableBytesWritable minKeyPtr = new ImmutableBytesWritable();
 
     private StatisticsWriter(HTableInterface statsReaderTable, HTableInterface statsWriterTable, String tableName, long clientTimeStamp) {
         this.statsReaderTable = statsReaderTable;
@@ -110,10 +115,19 @@ public class StatisticsWriter implements Closeable {
         long readTimeStamp = useMaxTimeStamp ? HConstants.LATEST_TIMESTAMP : clientTimeStamp;
         Result result = StatisticsUtil.readRegionStatistics(statsReaderTable, tableName, cfKey, p.getRegionName(),
                 readTimeStamp);
+        byte[] minKey = HConstants.EMPTY_BYTE_ARRAY;
         if (result != null && !result.isEmpty()) {
         	Cell cell = result.getColumnLatestCell(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, PhoenixDatabaseMetaData.GUIDE_POSTS_BYTES);
         	Cell rowCountCell = result.getColumnLatestCell(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, PhoenixDatabaseMetaData.GUIDE_POSTS_ROW_COUNT_BYTES);
+            Cell minKeyCell =
+                    result.getColumnLatestCell(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES,
+                        PhoenixDatabaseMetaData.MIN_KEY_BYTES);
             long rowCount = 0;
+            if (minKeyCell != null) {
+                minKey =
+                        ByteUtil.copyKeyBytesIfNecessary(new ImmutableBytesWritable(minKeyCell.getValueArray(),
+                                minKeyCell.getValueOffset(), minKeyCell.getValueLength()));
+            }
         	if (cell != null) {
                 long writeTimeStamp = useMaxTimeStamp ? cell.getTimestamp() : clientTimeStamp;
 
@@ -124,7 +138,7 @@ public class StatisticsWriter implements Closeable {
                 
 	        	long byteSize = 0;
                 Cell byteSizeCell = result.getColumnLatestCell(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES,
-                        PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH_BYTES);
+                        PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH_BYTES);                
                 int index = Collections.binarySearch(guidePostsRegionInfo.getGuidePosts(), r.getStartKey(),
                         Bytes.BYTES_COMPARATOR);
                 int size = guidePostsRegionInfo.getGuidePosts().size();
@@ -155,24 +169,27 @@ public class StatisticsWriter implements Closeable {
                     rightByteCount = (long)((1 - per) * byteSize);
                 }
 	            if (midEndIndex > 0) {
-	                GuidePostsInfo lguidePosts = new GuidePostsInfo(leftByteCount, guidePostsRegionInfo
-                            .getGuidePosts().subList(0, midEndIndex), leftRowCount);
+                    GuidePostsInfo lguidePosts =
+                            new GuidePostsInfo(leftByteCount, guidePostsRegionInfo.getGuidePosts()
+                                    .subList(0, midEndIndex), leftRowCount);
                     tracker.clear();
-	                tracker.addGuidePost(cfKey, lguidePosts, leftByteCount, cell.getTimestamp());
+                    tracker.addGuidePost(cfKey, lguidePosts, leftByteCount, cell.getTimestamp(),
+                        minKey);
 	                addStats(l.getRegionName(), tracker, cfKey, mutations);
 	            }
 	            if (midStartIndex < size) {
-	                GuidePostsInfo rguidePosts = new GuidePostsInfo(rightByteCount, guidePostsRegionInfo
-                            .getGuidePosts().subList(midStartIndex, size),
-                            rightRowCount);
+                    GuidePostsInfo rguidePosts =
+                            new GuidePostsInfo(rightByteCount, guidePostsRegionInfo.getGuidePosts()
+                                    .subList(midStartIndex, size), rightRowCount);
 	                tracker.clear();
-	                tracker.addGuidePost(cfKey, rguidePosts, rightByteCount, cell.getTimestamp());
+                    tracker.addGuidePost(cfKey, rguidePosts, rightByteCount, cell.getTimestamp(),
+                        guidePostsRegionInfo.getGuidePosts().get(midStartIndex));
 	                addStats(r.getRegionName(), tracker, cfKey, mutations);
 	            }
         	}
         }
     }
-    
+
     /**
      * Update a list of statistics for a given region.  If the UPDATE STATISTICS <tablename> query is issued
      * then we use Upsert queries to update the table
@@ -184,6 +201,7 @@ public class StatisticsWriter implements Closeable {
      *             if we fail to do any of the puts. Any single failure will prevent any future attempts for the remaining list of stats to
      *             update
      */
+    @SuppressWarnings("deprecation")
     public void addStats(byte[] regionName, StatisticsCollector tracker, ImmutableBytesPtr cfKey,
             List<Mutation> mutations) throws IOException {
         if (tracker == null) { return; }
@@ -206,6 +224,11 @@ public class StatisticsWriter implements Closeable {
             // Write as long_array
             put.add(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, PhoenixDatabaseMetaData.GUIDE_POSTS_ROW_COUNT_BYTES,
                     timeStamp, PLong.INSTANCE.toBytes(gp.getRowCount()));
+            tracker.getMinKey(minKeyPtr);
+            put.add(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES,
+                PhoenixDatabaseMetaData.MIN_KEY_BYTES, timeStamp,
+                // Ideally no copy would happen here
+                PVarbinary.INSTANCE.toBytes(ByteUtil.copyKeyBytesIfNecessary(minKeyPtr)));
         }
         // Add our empty column value so queries behave correctly
         put.add(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, QueryConstants.EMPTY_COLUMN_BYTES, timeStamp,