You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by td...@apache.org on 2015/06/24 22:24:15 UTC

[19/49] phoenix git commit: PHOENIX-1681 Use the new Region Interface (Andrew Purtell)

http://git-wip-us.apache.org/repos/asf/phoenix/blob/edff624f/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 272cac6..e7e1dd7 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
@@ -31,8 +31,8 @@ import org.apache.hadoop.hbase.KeyValueUtil;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
+import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.Store;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.coprocessor.MetaDataProtocol;
@@ -111,7 +111,7 @@ public class StatisticsCollector {
         this.statsTable.close();
     }
 
-    public void updateStatistic(HRegion region) {
+    public void updateStatistic(Region region) {
         try {
             ArrayList<Mutation> mutations = new ArrayList<Mutation>();
             writeStatsToStatsTable(region, true, mutations, TimeKeeper.SYSTEM.getCurrentTime());
@@ -126,7 +126,7 @@ public class StatisticsCollector {
         }
     }
     
-    private void writeStatsToStatsTable(final HRegion region,
+    private void writeStatsToStatsTable(final Region region,
             boolean delete, List<Mutation> mutations, long currentTime) throws IOException {
         try {
             // update the statistics table
@@ -215,7 +215,7 @@ public class StatisticsCollector {
         }
     }
 
-    public InternalScanner createCompactionScanner(HRegion region, Store store, InternalScanner s) throws IOException {
+    public InternalScanner createCompactionScanner(Region region, Store store, InternalScanner s) throws IOException {
         // See if this is for Major compaction
         if (logger.isDebugEnabled()) {
             logger.debug("Compaction scanner created for stats");
@@ -224,13 +224,13 @@ public class StatisticsCollector {
         return getInternalScanner(region, store, s, cfKey);
     }
 
-    public void splitStats(HRegion parent, HRegion left, HRegion right) {
+    public void splitStats(Region parent, Region left, Region right) {
         try {
             if (logger.isDebugEnabled()) {
                 logger.debug("Collecting stats for split of " + parent.getRegionInfo() + " into " + left.getRegionInfo() + " and " + right.getRegionInfo());
             }
             List<Mutation> mutations = Lists.newArrayListWithExpectedSize(3);
-            for (byte[] fam : parent.getStores().keySet()) {
+            for (byte[] fam : parent.getTableDesc().getFamiliesKeys()) {
             	statsTable.splitStats(parent, left, right, this, new ImmutableBytesPtr(fam), mutations);
             }
             if (logger.isDebugEnabled()) {
@@ -243,7 +243,7 @@ public class StatisticsCollector {
         }
     }
 
-    protected InternalScanner getInternalScanner(HRegion region, Store store,
+    protected InternalScanner getInternalScanner(Region region, Store store,
             InternalScanner internalScan, ImmutableBytesPtr family) {
         return new StatisticsScanner(this, statsTable, region, internalScan, family);
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/edff624f/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsScanner.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsScanner.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsScanner.java
index 0e50923..582c4de 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsScanner.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsScanner.java
@@ -26,9 +26,9 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.InternalScanner;
 import org.apache.hadoop.hbase.regionserver.ScannerContext;
+import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 
 /**
@@ -38,11 +38,11 @@ public class StatisticsScanner implements InternalScanner {
     private static final Log LOG = LogFactory.getLog(StatisticsScanner.class);
     private InternalScanner delegate;
     private StatisticsWriter stats;
-    private HRegion region;
+    private Region region;
     private StatisticsCollector tracker;
     private ImmutableBytesPtr family;
 
-    public StatisticsScanner(StatisticsCollector tracker, StatisticsWriter stats, HRegion region,
+    public StatisticsScanner(StatisticsCollector tracker, StatisticsWriter stats, Region region,
             InternalScanner delegate, ImmutableBytesPtr family) {
         this.tracker = tracker;
         this.stats = stats;
@@ -85,17 +85,17 @@ public class StatisticsScanner implements InternalScanner {
             // Just verify if this if fine
             ArrayList<Mutation> mutations = new ArrayList<Mutation>();
             if (LOG.isDebugEnabled()) {
-                LOG.debug("Deleting the stats for the region " + region.getRegionNameAsString()
+                LOG.debug("Deleting the stats for the region " + region.getRegionInfo().getRegionNameAsString()
                         + " as part of major compaction");
             }
-            stats.deleteStats(region.getRegionName(), this.tracker, family, mutations);
+            stats.deleteStats(region.getRegionInfo().getRegionName(), this.tracker, family, mutations);
             if (LOG.isDebugEnabled()) {
-                LOG.debug("Adding new stats for the region " + region.getRegionNameAsString()
+                LOG.debug("Adding new stats for the region " + region.getRegionInfo().getRegionNameAsString()
                         + " as part of major compaction");
             }
-            stats.addStats(region.getRegionName(), this.tracker, family, mutations);
+            stats.addStats(region.getRegionInfo().getRegionName(), this.tracker, family, mutations);
             if (LOG.isDebugEnabled()) {
-                LOG.debug("Committing new stats for the region " + region.getRegionNameAsString()
+                LOG.debug("Committing new stats for the region " + region.getRegionInfo().getRegionNameAsString()
                         + " as part of major compaction");
             }
             stats.commitStats(mutations);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/edff624f/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 8756568..834675c 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
@@ -42,7 +42,7 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
 import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MultiRowMutationService;
 import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsRequest;
-import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
@@ -105,7 +105,7 @@ public class StatisticsWriter implements Closeable {
         statsWriterTable.close();
     }
 
-    public void splitStats(HRegion p, HRegion l, HRegion r, StatisticsCollector tracker, ImmutableBytesPtr cfKey,
+    public void splitStats(Region p, Region l, Region r, StatisticsCollector tracker, ImmutableBytesPtr cfKey,
             List<Mutation> mutations) throws IOException {
         if (tracker == null) { return; }
         boolean useMaxTimeStamp = clientTimeStamp == StatisticsCollector.NO_TIMESTAMP;
@@ -113,8 +113,8 @@ public class StatisticsWriter implements Closeable {
             mutations.add(getLastStatsUpdatedTimePut(clientTimeStamp));
         }
         long readTimeStamp = useMaxTimeStamp ? HConstants.LATEST_TIMESTAMP : clientTimeStamp;
-        Result result = StatisticsUtil.readRegionStatistics(statsReaderTable, tableName, cfKey, p.getRegionName(),
-                readTimeStamp);
+        Result result = StatisticsUtil.readRegionStatistics(statsReaderTable, tableName, cfKey, 
+                p.getRegionInfo().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);
@@ -133,13 +133,13 @@ public class StatisticsWriter implements Closeable {
 
                 GuidePostsInfo guidePostsRegionInfo = GuidePostsInfo.deserializeGuidePostsInfo(cell.getValueArray(),
                         cell.getValueOffset(), cell.getValueLength(), rowCount);
-                byte[] pPrefix = StatisticsUtil.getRowKey(tableName, cfKey, p.getRegionName());
+                byte[] pPrefix = StatisticsUtil.getRowKey(tableName, cfKey, p.getRegionInfo().getRegionName());
                 mutations.add(new Delete(pPrefix, writeTimeStamp));
                 
 	        	long byteSize = 0;
                 Cell byteSizeCell = result.getColumnLatestCell(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES,
                         PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH_BYTES);                
-                int index = Collections.binarySearch(guidePostsRegionInfo.getGuidePosts(), r.getStartKey(),
+                int index = Collections.binarySearch(guidePostsRegionInfo.getGuidePosts(), r.getRegionInfo().getStartKey(),
                         Bytes.BYTES_COMPARATOR);
                 int size = guidePostsRegionInfo.getGuidePosts().size();
                 int midEndIndex, midStartIndex;
@@ -175,7 +175,7 @@ public class StatisticsWriter implements Closeable {
                     tracker.clear();
                     tracker.addGuidePost(cfKey, lguidePosts, leftByteCount, cell.getTimestamp(),
                         minKey);
-	                addStats(l.getRegionName(), tracker, cfKey, mutations);
+	                addStats(l.getRegionInfo().getRegionName(), tracker, cfKey, mutations);
 	            }
 	            if (midStartIndex < size) {
                     GuidePostsInfo rguidePosts =
@@ -184,7 +184,7 @@ public class StatisticsWriter implements Closeable {
 	                tracker.clear();
                     tracker.addGuidePost(cfKey, rguidePosts, rightByteCount, cell.getTimestamp(),
                         guidePostsRegionInfo.getGuidePosts().get(midStartIndex));
-	                addStats(r.getRegionName(), tracker, cfKey, mutations);
+	                addStats(r.getRegionInfo().getRegionName(), tracker, cfKey, mutations);
 	            }
         	}
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/edff624f/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
index ca25348..3bf6f23 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
@@ -40,7 +40,7 @@ import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessorEnvironment;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.WritableUtils;
@@ -305,47 +305,49 @@ public class IndexUtil {
         });
     }
 
-    public static HRegion getIndexRegion(RegionCoprocessorEnvironment environment)
+    public static Region getIndexRegion(RegionCoprocessorEnvironment environment)
             throws IOException {
-        HRegion dataRegion = environment.getRegion();
+        Region dataRegion = environment.getRegion();
         return getIndexRegion(dataRegion, environment.getRegionServerServices());
     }
 
-    public static HRegion
-            getIndexRegion(HRegion dataRegion, RegionServerCoprocessorEnvironment env)
+    public static Region
+            getIndexRegion(Region dataRegion, RegionServerCoprocessorEnvironment env)
                     throws IOException {
         return getIndexRegion(dataRegion, env.getRegionServerServices());
     }
 
-    public static HRegion getDataRegion(RegionCoprocessorEnvironment env) throws IOException {
-        HRegion indexRegion = env.getRegion();
+    public static Region getDataRegion(RegionCoprocessorEnvironment env) throws IOException {
+        Region indexRegion = env.getRegion();
         return getDataRegion(indexRegion, env.getRegionServerServices());
     }
 
-    public static HRegion
-            getDataRegion(HRegion indexRegion, RegionServerCoprocessorEnvironment env)
+    public static Region
+            getDataRegion(Region indexRegion, RegionServerCoprocessorEnvironment env)
                     throws IOException {
         return getDataRegion(indexRegion, env.getRegionServerServices());
     }
 
-    public static HRegion getIndexRegion(HRegion dataRegion, RegionServerServices rss) throws IOException {
+    public static Region getIndexRegion(Region dataRegion, RegionServerServices rss) throws IOException {
         TableName indexTableName =
                 TableName.valueOf(MetaDataUtil.getLocalIndexPhysicalName(dataRegion.getTableDesc()
                         .getName()));
-        List<HRegion> onlineRegions = rss.getOnlineRegions(indexTableName);
-        for(HRegion indexRegion : onlineRegions) {
-            if (Bytes.compareTo(dataRegion.getStartKey(), indexRegion.getStartKey()) == 0) {
+        List<Region> onlineRegions = rss.getOnlineRegions(indexTableName);
+        for(Region indexRegion : onlineRegions) {
+            if (Bytes.compareTo(dataRegion.getRegionInfo().getStartKey(),
+                    indexRegion.getRegionInfo().getStartKey()) == 0) {
                 return indexRegion;
             }
         }
         return null;
     }
 
-    public static HRegion getDataRegion(HRegion indexRegion, RegionServerServices rss) throws IOException {
+    public static Region getDataRegion(Region indexRegion, RegionServerServices rss) throws IOException {
         TableName dataTableName = TableName.valueOf(MetaDataUtil.getUserTableName(indexRegion.getTableDesc().getNameAsString()));
-        List<HRegion> onlineRegions = rss.getOnlineRegions(dataTableName);
-        for(HRegion region : onlineRegions) {
-            if (Bytes.compareTo(indexRegion.getStartKey(), region.getStartKey()) == 0) {
+        List<Region> onlineRegions = rss.getOnlineRegions(dataTableName);
+        for(Region region : onlineRegions) {
+            if (Bytes.compareTo(indexRegion.getRegionInfo().getStartKey(),
+                    region.getRegionInfo().getStartKey()) == 0) {
                 return region;
             }
         }
@@ -466,7 +468,7 @@ public class IndexUtil {
     
     public static void wrapResultUsingOffset(final ObserverContext<RegionCoprocessorEnvironment> c,
             List<Cell> result, final int offset, ColumnReference[] dataColumns,
-            TupleProjector tupleProjector, HRegion dataRegion, IndexMaintainer indexMaintainer,
+            TupleProjector tupleProjector, Region dataRegion, IndexMaintainer indexMaintainer,
             byte[][] viewConstants, ImmutableBytesWritable ptr) throws IOException {
         if (tupleProjector != null) {
             // Join back to data table here by issuing a local get projecting

http://git-wip-us.apache.org/repos/asf/phoenix/blob/edff624f/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/TestLocalTableState.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/TestLocalTableState.java b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/TestLocalTableState.java
index e996b23..fa8bd85 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/TestLocalTableState.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/TestLocalTableState.java
@@ -29,7 +29,7 @@ import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
@@ -65,7 +65,7 @@ public class TestLocalTableState {
     RegionCoprocessorEnvironment env = Mockito.mock(RegionCoprocessorEnvironment.class);
     Mockito.when(env.getConfiguration()).thenReturn(conf);
 
-    HRegion region = Mockito.mock(HRegion.class);
+    Region region = Mockito.mock(Region.class);
     Mockito.when(env.getRegion()).thenReturn(region);
     RegionScanner scanner = Mockito.mock(RegionScanner.class);
     Mockito.when(region.getScanner(Mockito.any(Scan.class))).thenReturn(scanner);
@@ -108,7 +108,7 @@ public class TestLocalTableState {
     // setup mocks
     RegionCoprocessorEnvironment env = Mockito.mock(RegionCoprocessorEnvironment.class);
 
-    HRegion region = Mockito.mock(HRegion.class);
+    Region region = Mockito.mock(Region.class);
     Mockito.when(env.getRegion()).thenReturn(region);
     RegionScanner scanner = Mockito.mock(RegionScanner.class);
     Mockito.when(region.getScanner(Mockito.any(Scan.class))).thenReturn(scanner);
@@ -154,7 +154,7 @@ public class TestLocalTableState {
     // setup mocks
     RegionCoprocessorEnvironment env = Mockito.mock(RegionCoprocessorEnvironment.class);
 
-    HRegion region = Mockito.mock(HRegion.class);
+    Region region = Mockito.mock(Region.class);
     Mockito.when(env.getRegion()).thenReturn(region);
     RegionScanner scanner = Mockito.mock(RegionScanner.class);
     Mockito.when(region.getScanner(Mockito.any(Scan.class))).thenReturn(scanner);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/edff624f/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestWALRecoveryCaching.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestWALRecoveryCaching.java b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestWALRecoveryCaching.java
index ae577bd..b381e9f 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestWALRecoveryCaching.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestWALRecoveryCaching.java
@@ -50,8 +50,8 @@ import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
+import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -201,7 +201,7 @@ public class TestWALRecoveryCaching {
 
     // kill the server where the tables live - this should trigger distributed log splitting
     // find the regionserver that matches the passed server
-    List<HRegion> online = new ArrayList<HRegion>();
+    List<Region> online = new ArrayList<Region>();
     online.addAll(getRegionsFromServerForTable(util.getMiniHBaseCluster(), shared,
       testTable.getTableName()));
     online.addAll(getRegionsFromServerForTable(util.getMiniHBaseCluster(), shared,
@@ -267,9 +267,9 @@ public class TestWALRecoveryCaching {
    * @param table
    * @return
    */
-  private List<HRegion> getRegionsFromServerForTable(MiniHBaseCluster cluster, ServerName server,
+  private List<Region> getRegionsFromServerForTable(MiniHBaseCluster cluster, ServerName server,
       byte[] table) {
-    List<HRegion> online = Collections.emptyList();
+    List<Region> online = Collections.emptyList();
     for (RegionServerThread rst : cluster.getRegionServerThreads()) {
       // if its the server we are going to kill, get the regions we want to reassign
       if (rst.getRegionServer().getServerName().equals(server)) {
@@ -305,14 +305,14 @@ public class TestWALRecoveryCaching {
       tryIndex = !tryIndex;
       for (ServerName server : servers) {
         // find the regionserver that matches the passed server
-        List<HRegion> online = getRegionsFromServerForTable(cluster, server, table);
+        List<Region> online = getRegionsFromServerForTable(cluster, server, table);
 
         LOG.info("Shutting down and reassigning regions from " + server);
         cluster.stopRegionServer(server);
         cluster.waitForRegionServerToStop(server, TIMEOUT);
 
         // force reassign the regions from the table
-        for (HRegion region : online) {
+        for (Region region : online) {
           cluster.getMaster().assignRegion(region.getRegionInfo());
         }
 
@@ -363,10 +363,9 @@ public class TestWALRecoveryCaching {
 
   private Set<ServerName> getServersForTable(MiniHBaseCluster cluster, byte[] table)
       throws Exception {
-    List<HRegion> indexRegions = cluster.getRegions(table);
     Set<ServerName> indexServers = new HashSet<ServerName>();
-    for (HRegion region : indexRegions) {
-      indexServers.add(cluster.getServerHoldingRegion(null, region.getRegionName()));
+    for (Region region : cluster.getRegions(table)) {
+      indexServers.add(cluster.getServerHoldingRegion(null, region.getRegionInfo().getRegionName()));
     }
     return indexServers;
   }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/edff624f/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/recovery/TestPerRegionIndexWriteCache.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/recovery/TestPerRegionIndexWriteCache.java b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/recovery/TestPerRegionIndexWriteCache.java
index cd28627..35b607e 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/recovery/TestPerRegionIndexWriteCache.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/recovery/TestPerRegionIndexWriteCache.java
@@ -65,8 +65,8 @@ public class TestPerRegionIndexWriteCache {
     p2.add(family, qual, val);
   }
 
-  HRegion r1;
-  HRegion r2;
+  HRegion r1; // FIXME: Uses private type
+  HRegion r2; // FIXME: Uses private type
   WAL wal;
 
   @SuppressWarnings("deprecation")
@@ -212,4 +212,4 @@ public class TestPerRegionIndexWriteCache {
     // references around to these edits and have a memory leak
     assertNull("Got an entry for a region we removed", cache.getEdits(r1));
   }
-}
\ No newline at end of file
+}