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

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

Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-1.1 [created] 980d29c5a


http://git-wip-us.apache.org/repos/asf/phoenix/blob/ea622d5f/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/ea622d5f/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/ea622d5f/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/ea622d5f/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/ea622d5f/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/ea622d5f/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/ea622d5f/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
+}


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

Posted by nd...@apache.org.
PHOENIX-1681 Use the new Region Interface (Andrew Purtell)


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

Branch: refs/heads/4.x-HBase-1.1
Commit: ea622d5f7ab5c37d2ecf8be6054e5ed42f36a035
Parents: 98271b8
Author: Enis Soztutar <en...@apache.org>
Authored: Thu May 21 23:22:54 2015 -0700
Committer: Enis Soztutar <en...@apache.org>
Committed: Fri May 22 00:30:56 2015 -0700

----------------------------------------------------------------------
 ...ReplayWithIndexWritesAndCompressedWALIT.java |  4 +-
 .../EndToEndCoveredColumnsIndexBuilderIT.java   |  4 +-
 .../IndexHalfStoreFileReaderGenerator.java      |  9 +-
 .../regionserver/IndexSplitTransaction.java     | 65 +++++---------
 .../hbase/regionserver/LocalIndexMerger.java    | 16 ++--
 .../hbase/regionserver/LocalIndexSplitter.java  | 11 +--
 .../coprocessor/BaseScannerRegionObserver.java  | 26 +++---
 .../GroupedAggregateRegionObserver.java         | 13 +--
 .../coprocessor/MetaDataEndpointImpl.java       | 94 ++++++++++----------
 .../phoenix/coprocessor/ScanRegionObserver.java | 17 ++--
 .../coprocessor/SequenceRegionObserver.java     | 16 ++--
 .../UngroupedAggregateRegionObserver.java       | 29 +++---
 .../hbase/index/covered/data/LocalTable.java    |  5 +-
 .../write/ParallelWriterIndexCommitter.java     |  8 +-
 .../recovery/PerRegionIndexWriteCache.java      | 10 +--
 .../recovery/StoreFailuresInCachePolicy.java    |  4 +-
 .../TrackingParallelWriterIndexCommitter.java   |  8 +-
 .../phoenix/index/PhoenixIndexBuilder.java      |  4 +-
 .../apache/phoenix/index/PhoenixIndexCodec.java | 14 ++-
 .../schema/stats/StatisticsCollector.java       | 14 +--
 .../phoenix/schema/stats/StatisticsScanner.java | 16 ++--
 .../phoenix/schema/stats/StatisticsWriter.java  | 16 ++--
 .../java/org/apache/phoenix/util/IndexUtil.java | 38 ++++----
 .../index/covered/TestLocalTableState.java      |  8 +-
 .../index/write/TestWALRecoveryCaching.java     | 17 ++--
 .../recovery/TestPerRegionIndexWriteCache.java  |  6 +-
 26 files changed, 230 insertions(+), 242 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/ea622d5f/phoenix-core/src/it/java/org/apache/hadoop/hbase/regionserver/wal/WALReplayWithIndexWritesAndCompressedWALIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/hadoop/hbase/regionserver/wal/WALReplayWithIndexWritesAndCompressedWALIT.java b/phoenix-core/src/it/java/org/apache/hadoop/hbase/regionserver/wal/WALReplayWithIndexWritesAndCompressedWALIT.java
index 3b8ff29..611ba68 100644
--- a/phoenix-core/src/it/java/org/apache/hadoop/hbase/regionserver/wal/WALReplayWithIndexWritesAndCompressedWALIT.java
+++ b/phoenix-core/src/it/java/org/apache/hadoop/hbase/regionserver/wal/WALReplayWithIndexWritesAndCompressedWALIT.java
@@ -159,7 +159,7 @@ public class WALReplayWithIndexWritesAndCompressedWALIT {
   }
 
   /**
-   * Test writing edits into an HRegion, closing it, splitting logs, opening Region again. Verify
+   * Test writing edits into an region, closing it, splitting logs, opening Region again. Verify
    * seqids.
    * @throws Exception on failure
    */
@@ -183,7 +183,7 @@ public class WALReplayWithIndexWritesAndCompressedWALIT {
     builder.build(htd);
 
     // create the region + its WAL
-    HRegion region0 = HRegion.createHRegion(hri, hbaseRootDir, this.conf, htd);
+    HRegion region0 = HRegion.createHRegion(hri, hbaseRootDir, this.conf, htd); // FIXME: Uses private type
     region0.close();
     region0.getWAL().close();
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ea622d5f/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/covered/EndToEndCoveredColumnsIndexBuilderIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/covered/EndToEndCoveredColumnsIndexBuilderIT.java b/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/covered/EndToEndCoveredColumnsIndexBuilderIT.java
index d90733f..6b2309e 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/covered/EndToEndCoveredColumnsIndexBuilderIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/covered/EndToEndCoveredColumnsIndexBuilderIT.java
@@ -41,7 +41,7 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
-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.util.EnvironmentEdge;
 import org.apache.phoenix.util.EnvironmentEdgeManager;
@@ -312,7 +312,7 @@ public class EndToEndCoveredColumnsIndexBuilderIT {
     HTable primary = new HTable(UTIL.getConfiguration(), tableNameBytes);
 
     // overwrite the codec so we can verify the current state
-    HRegion region = UTIL.getMiniHBaseCluster().getRegions(tableNameBytes).get(0);
+    Region region = UTIL.getMiniHBaseCluster().getRegions(tableNameBytes).get(0);
     Indexer indexer =
         (Indexer) region.getCoprocessorHost().findCoprocessor(Indexer.class.getName());
     CoveredColumnsIndexBuilder builder =

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ea622d5f/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReaderGenerator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReaderGenerator.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReaderGenerator.java
index 1284dcf..94d5912 100644
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReaderGenerator.java
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReaderGenerator.java
@@ -76,7 +76,7 @@ public class IndexHalfStoreFileReaderGenerator extends BaseRegionObserver {
             FileSystem fs, Path p, FSDataInputStreamWrapper in, long size, CacheConfig cacheConf,
             Reference r, Reader reader) throws IOException {
         TableName tableName = ctx.getEnvironment().getRegion().getTableDesc().getTableName();
-        HRegion region = ctx.getEnvironment().getRegion();
+        Region region = ctx.getEnvironment().getRegion();
         HRegionInfo childRegion = region.getRegionInfo();
         byte[] splitKey = null;
         if (reader == null && r != null) {
@@ -109,7 +109,7 @@ public class IndexHalfStoreFileReaderGenerator extends BaseRegionObserver {
                     Pair<HRegionInfo, HRegionInfo> mergeRegions =
                             MetaTableAccessor.getRegionsFromMergeQualifier(ctx.getEnvironment()
                                     .getRegionServerServices().getConnection(),
-                                region.getRegionName());
+                                region.getRegionInfo().getRegionName());
                     if (mergeRegions == null || mergeRegions.getFirst() == null) return reader;
                     byte[] splitRow =
                             CellUtil.cloneRow(KeyValue.createKeyValueFromKey(r.getSplitKey()));
@@ -121,8 +121,9 @@ public class IndexHalfStoreFileReaderGenerator extends BaseRegionObserver {
                         childRegion = mergeRegions.getSecond();
                         regionStartKeyInHFile = mergeRegions.getSecond().getStartKey();
                     }
-                    splitKey = KeyValue.createFirstOnRow(region.getStartKey().length == 0 ?
-                            new byte[region.getEndKey().length] : region.getStartKey()).getKey();
+                    splitKey = KeyValue.createFirstOnRow(region.getRegionInfo().getStartKey().length == 0 ?
+                        new byte[region.getRegionInfo().getEndKey().length] :
+                            region.getRegionInfo().getStartKey()).getKey();
                 } else {
                     HRegionInfo parentRegion = HRegionInfo.getHRegionInfo(result);
                     regionStartKeyInHFile =

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ea622d5f/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexSplitTransaction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexSplitTransaction.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexSplitTransaction.java
index 3057a14..71bc520 100644
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexSplitTransaction.java
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexSplitTransaction.java
@@ -65,31 +65,8 @@ import org.apache.zookeeper.data.Stat;
 
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 
-/**
- * Executes region split as a "transaction".  Call {@link #prepare()} to setup
- * the transaction, {@link #execute(Server, RegionServerServices)} to run the
- * transaction and {@link #rollback(Server, RegionServerServices)} to cleanup if execute fails.
- *
- * <p>Here is an example of how you would use this class:
- * <pre>
- *  SplitTransaction st = new SplitTransaction(this.conf, parent, midKey)
- *  if (!st.prepare()) return;
- *  try {
- *    st.execute(server, services);
- *  } catch (IOException ioe) {
- *    try {
- *      st.rollback(server, services);
- *      return;
- *    } catch (RuntimeException e) {
- *      myAbortable.abort("Failed split, abort");
- *    }
- *  }
- * </Pre>
- * <p>This class is not thread safe.  Caller needs ensure split is run by
- * one thread only.
- */
 @InterfaceAudience.Private
-public class IndexSplitTransaction extends SplitTransaction {
+public class IndexSplitTransaction extends SplitTransactionImpl { // FIXME: Extends private type
   private static final Log LOG = LogFactory.getLog(IndexSplitTransaction.class);
 
   /*
@@ -154,9 +131,9 @@ public class IndexSplitTransaction extends SplitTransaction {
    * @param r Region to split
    * @param splitrow Row to split around
    */
-  public IndexSplitTransaction(final HRegion r, final byte [] splitrow) {
+  public IndexSplitTransaction(final Region r, final byte [] splitrow) {
     super(r , splitrow);
-    this.parent = r;
+    this.parent = (HRegion)r;
     this.splitrow = splitrow;
   }
 
@@ -217,7 +194,7 @@ public class IndexSplitTransaction extends SplitTransaction {
    * @return Regions created
    */
   @Override
-  /* package */PairOfSameType<HRegion> createDaughters(final Server server,
+  /* package */PairOfSameType<Region> createDaughters(final Server server,
       final RegionServerServices services) throws IOException {
     LOG.info("Starting split of region " + this.parent);
     if ((server != null && server.isStopped()) ||
@@ -244,14 +221,14 @@ public class IndexSplitTransaction extends SplitTransaction {
         server.getConfiguration().getLong("hbase.regionserver.fileSplitTimeout",
           this.fileSplitTimeout);
 
-    PairOfSameType<HRegion> daughterRegions = stepsBeforePONR(server, services, testing);
+    PairOfSameType<Region> daughterRegions = stepsBeforePONR(server, services, testing);
 
     List<Mutation> metaEntries = new ArrayList<Mutation>();
     if (this.parent.getCoprocessorHost() != null) {
       if (this.parent.getCoprocessorHost().
           preSplitBeforePONR(this.splitrow, metaEntries)) {
         throw new IOException("Coprocessor bypassing region "
-            + this.parent.getRegionNameAsString() + " split.");
+            + this.parent.getRegionInfo().getRegionNameAsString() + " split.");
       }
       try {
         for (Mutation p : metaEntries) {
@@ -303,7 +280,7 @@ public class IndexSplitTransaction extends SplitTransaction {
   }
 
   @Override
-  public PairOfSameType<HRegion> stepsBeforePONR(final Server server,
+  public PairOfSameType<Region> stepsBeforePONR(final Server server,
       final RegionServerServices services, boolean testing) throws IOException {
     // Set ephemeral SPLITTING znode up in zk.  Mocked servers sometimes don't
     // have zookeeper so don't do zk stuff if server or zookeeper is null
@@ -313,7 +290,7 @@ public class IndexSplitTransaction extends SplitTransaction {
           parent.getRegionInfo(), server.getServerName(), hri_a, hri_b);
       } catch (KeeperException e) {
         throw new IOException("Failed creating PENDING_SPLIT znode on " +
-          this.parent.getRegionNameAsString(), e);
+          this.parent.getRegionInfo().getRegionNameAsString(), e);
       }
     }
     this.journal.add(JournalEntry.SET_SPLITTING_IN_ZK);
@@ -367,12 +344,12 @@ public class IndexSplitTransaction extends SplitTransaction {
     // stuff in fs that needs cleanup -- a storefile or two.  Thats why we
     // add entry to journal BEFORE rather than AFTER the change.
     this.journal.add(JournalEntry.STARTED_REGION_A_CREATION);
-    HRegion a = this.parent.createDaughterRegionFromSplits(this.hri_a);
+    Region a = this.parent.createDaughterRegionFromSplits(this.hri_a);
 
     // Ditto
     this.journal.add(JournalEntry.STARTED_REGION_B_CREATION);
-    HRegion b = this.parent.createDaughterRegionFromSplits(this.hri_b);
-    return new PairOfSameType<HRegion>(a, b);
+    Region b = this.parent.createDaughterRegionFromSplits(this.hri_b);
+    return new PairOfSameType<Region>(a, b);
   }
 
   /**
@@ -387,7 +364,7 @@ public class IndexSplitTransaction extends SplitTransaction {
    */
   @Override
   /* package */void openDaughters(final Server server,
-      final RegionServerServices services, HRegion a, HRegion b)
+      final RegionServerServices services, Region a, Region b)
       throws IOException {
     boolean stopped = server != null && server.isStopped();
     boolean stopping = services != null && services.isStopping();
@@ -400,8 +377,8 @@ public class IndexSplitTransaction extends SplitTransaction {
           " because stopping=" + stopping + ", stopped=" + stopped);
     } else {
       // Open daughters in parallel.
-      DaughterOpener aOpener = new DaughterOpener(server, a);
-      DaughterOpener bOpener = new DaughterOpener(server, b);
+      DaughterOpener aOpener = new DaughterOpener(server, (HRegion)a);
+      DaughterOpener bOpener = new DaughterOpener(server, (HRegion)b);
       aOpener.start();
       bOpener.start();
       try {
@@ -444,7 +421,7 @@ public class IndexSplitTransaction extends SplitTransaction {
    *          Call {@link #rollback(Server, RegionServerServices)}
    */
   /* package */void transitionZKNode(final Server server,
-      final RegionServerServices services, HRegion a, HRegion b)
+      final RegionServerServices services, Region a, Region b)
       throws IOException {
     // Tell master about split by updating zk.  If we fail, abort.
     if (server != null && server.getZooKeeper() != null) {
@@ -556,7 +533,7 @@ public class IndexSplitTransaction extends SplitTransaction {
         Thread.currentThread().interrupt();
       }
       throw new IOException("Failed getting SPLITTING znode on "
-        + parent.getRegionNameAsString(), e);
+        + parent.getRegionInfo().getRegionNameAsString(), e);
     }
   }
 
@@ -572,10 +549,10 @@ public class IndexSplitTransaction extends SplitTransaction {
    * @see #rollback(Server, RegionServerServices)
    */
   @Override
-  public PairOfSameType<HRegion> execute(final Server server,
+  public PairOfSameType<Region> execute(final Server server,
       final RegionServerServices services)
   throws IOException {
-    PairOfSameType<HRegion> regions = createDaughters(server, services);
+    PairOfSameType<Region> regions = createDaughters(server, services);
     if (this.parent.getCoprocessorHost() != null) {
       this.parent.getCoprocessorHost().preSplitAfterPONR();
     }
@@ -583,8 +560,8 @@ public class IndexSplitTransaction extends SplitTransaction {
   }
 
   @Override
-  public PairOfSameType<HRegion> stepsAfterPONR(final Server server,
-      final RegionServerServices services, PairOfSameType<HRegion> regions)
+  public PairOfSameType<Region> stepsAfterPONR(final Server server,
+      final RegionServerServices services, PairOfSameType<Region> regions)
       throws IOException {
     openDaughters(server, services, regions.getFirst(), regions.getSecond());
     transitionZKNode(server, services, regions.getFirst(), regions.getSecond());
@@ -871,7 +848,7 @@ public class IndexSplitTransaction extends SplitTransaction {
           this.parent.initialize();
         } catch (IOException e) {
           LOG.error("Failed rollbacking CLOSED_PARENT_REGION of region " +
-            this.parent.getRegionNameAsString(), e);
+            this.parent.getRegionInfo().getRegionNameAsString(), e);
           throw new RuntimeException(e);
         }
         break;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ea622d5f/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/LocalIndexMerger.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/LocalIndexMerger.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/LocalIndexMerger.java
index add9b72..e361343 100644
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/LocalIndexMerger.java
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/LocalIndexMerger.java
@@ -38,12 +38,12 @@ public class LocalIndexMerger extends BaseRegionServerObserver {
 
     private static final Log LOG = LogFactory.getLog(LocalIndexMerger.class);
 
-    private RegionMergeTransaction rmt = null;
-    private HRegion mergedRegion = null;
+    private RegionMergeTransactionImpl rmt = null; // FIXME: Use of private type
+    private HRegion mergedRegion = null; // FIXME: Use of private type
 
     @Override
     public void preMergeCommit(ObserverContext<RegionServerCoprocessorEnvironment> ctx,
-            HRegion regionA, HRegion regionB, List<Mutation> metaEntries) throws IOException {
+            Region regionA, Region regionB, List<Mutation> metaEntries) throws IOException {
         HTableDescriptor tableDesc = regionA.getTableDesc();
         if (SchemaUtil.isSystemTable(tableDesc.getName())) {
             return;
@@ -56,14 +56,14 @@ public class LocalIndexMerger extends BaseRegionServerObserver {
             TableName indexTable =
                     TableName.valueOf(MetaDataUtil.getLocalIndexPhysicalName(tableDesc.getName()));
             if (!MetaTableAccessor.tableExists(rs.getConnection(), indexTable)) return;
-            HRegion indexRegionA = IndexUtil.getIndexRegion(regionA, ctx.getEnvironment());
+            Region indexRegionA = IndexUtil.getIndexRegion(regionA, ctx.getEnvironment());
             if (indexRegionA == null) {
                 LOG.warn("Index region corresponindg to data region " + regionA
                         + " not in the same server. So skipping the merge.");
                 ctx.bypass();
                 return;
             }
-            HRegion indexRegionB = IndexUtil.getIndexRegion(regionB, ctx.getEnvironment());
+            Region indexRegionB = IndexUtil.getIndexRegion(regionB, ctx.getEnvironment());
             if (indexRegionB == null) {
                 LOG.warn("Index region corresponindg to region " + regionB
                         + " not in the same server. So skipping the merge.");
@@ -71,7 +71,7 @@ public class LocalIndexMerger extends BaseRegionServerObserver {
                 return;
             }
             try {
-                rmt = new RegionMergeTransaction(indexRegionA, indexRegionB, false);
+                rmt = new RegionMergeTransactionImpl(indexRegionA, indexRegionB, false);
                 if (!rmt.prepare(rss)) {
                     LOG.error("Prepare for the index regions merge [" + indexRegionA + ","
                             + indexRegionB + "] failed. So returning null. ");
@@ -97,7 +97,7 @@ public class LocalIndexMerger extends BaseRegionServerObserver {
 
     @Override
     public void postMergeCommit(ObserverContext<RegionServerCoprocessorEnvironment> ctx,
-            HRegion regionA, HRegion regionB, HRegion mergedRegion) throws IOException {
+            Region regionA, Region regionB, Region mergedRegion) throws IOException {
         if (rmt != null && this.mergedRegion != null) {
             RegionServerCoprocessorEnvironment environment = ctx.getEnvironment();
             HRegionServer rs = (HRegionServer) environment.getRegionServerServices();
@@ -107,7 +107,7 @@ public class LocalIndexMerger extends BaseRegionServerObserver {
 
     @Override
     public void preRollBackMerge(ObserverContext<RegionServerCoprocessorEnvironment> ctx,
-            HRegion regionA, HRegion regionB) throws IOException {
+            Region regionA, Region regionB) throws IOException {
         HRegionServer rs = (HRegionServer) ctx.getEnvironment().getRegionServerServices();
         try {
             if (rmt != null) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ea622d5f/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/LocalIndexSplitter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/LocalIndexSplitter.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/LocalIndexSplitter.java
index 9af8251..7882e25 100644
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/LocalIndexSplitter.java
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/LocalIndexSplitter.java
@@ -52,8 +52,8 @@ public class LocalIndexSplitter extends BaseRegionObserver {
 
     private static final Log LOG = LogFactory.getLog(LocalIndexSplitter.class);
 
-    private SplitTransaction st = null;
-    private PairOfSameType<HRegion> daughterRegions = null;
+    private SplitTransactionImpl st = null; // FIXME: Uses private type
+    private PairOfSameType<Region> daughterRegions = null;
     private static final ParseNodeFactory FACTORY = new ParseNodeFactory();
     private static final int SPLIT_TXN_MINIMUM_SUPPORTED_VERSION = VersionUtil
             .encodeVersion("0.98.9");
@@ -74,17 +74,18 @@ public class LocalIndexSplitter extends BaseRegionObserver {
                     TableName.valueOf(MetaDataUtil.getLocalIndexPhysicalName(tableDesc.getName()));
             if (!MetaTableAccessor.tableExists(rss.getConnection(), indexTable)) return;
 
-            HRegion indexRegion = IndexUtil.getIndexRegion(environment);
+            Region indexRegion = IndexUtil.getIndexRegion(environment);
             if (indexRegion == null) {
                 LOG.warn("Index region corresponindg to data region " + environment.getRegion()
                         + " not in the same server. So skipping the split.");
                 ctx.bypass();
                 return;
             }
+            // FIXME: Uses private type
             try {
                 int encodedVersion = VersionUtil.encodeVersion(environment.getHBaseVersion());
                 if(encodedVersion >= SPLIT_TXN_MINIMUM_SUPPORTED_VERSION) {
-                    st = new SplitTransaction(indexRegion, splitKey);
+                    st = new SplitTransactionImpl(indexRegion, splitKey);
                     st.useZKForAssignment =
                             environment.getConfiguration().getBoolean("hbase.assignment.usezk",
                                 true);
@@ -98,7 +99,7 @@ public class LocalIndexSplitter extends BaseRegionObserver {
                     ctx.bypass();
                     return;
                 }
-                indexRegion.forceSplit(splitKey);
+                ((HRegion)indexRegion).forceSplit(splitKey);
                 daughterRegions = st.stepsBeforePONR(rss, rss, false);
                 HRegionInfo copyOfParent = new HRegionInfo(indexRegion.getRegionInfo());
                 copyOfParent.setOffline(true);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ea622d5f/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java
index fc74968..d9e64e2 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java
@@ -35,7 +35,7 @@ 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.io.ImmutableBytesWritable;
-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.regionserver.ScannerContext;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -114,12 +114,12 @@ abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
     }
 
 
-    private static void throwIfScanOutOfRegion(Scan scan, HRegion region) throws DoNotRetryIOException {
+    private static void throwIfScanOutOfRegion(Scan scan, Region region) throws DoNotRetryIOException {
         boolean isLocalIndex = ScanUtil.isLocalIndex(scan);
         byte[] lowerInclusiveScanKey = scan.getStartRow();
         byte[] upperExclusiveScanKey = scan.getStopRow();
-        byte[] lowerInclusiveRegionKey = region.getStartKey();
-        byte[] upperExclusiveRegionKey = region.getEndKey();
+        byte[] lowerInclusiveRegionKey = region.getRegionInfo().getStartKey();
+        byte[] upperExclusiveRegionKey = region.getRegionInfo().getEndKey();
         boolean isStaleRegionBoundaries;
         if (isLocalIndex) {
             byte[] expectedUpperRegionKey = scan.getAttribute(EXPECTED_UPPER_REGION_KEY);
@@ -201,7 +201,7 @@ abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
                 }
             }
         } catch (Throwable t) {
-            ServerUtil.throwIOException(c.getEnvironment().getRegion().getRegionNameAsString(), t);
+            ServerUtil.throwIOException(c.getEnvironment().getRegion().getRegionInfo().getRegionNameAsString(), t);
             return null; // impossible
         }
     }
@@ -221,7 +221,7 @@ abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
     protected RegionScanner getWrappedScanner(final ObserverContext<RegionCoprocessorEnvironment> c,
             final RegionScanner s, final int offset, final Scan scan,
             final ColumnReference[] dataColumns, final TupleProjector tupleProjector,
-            final HRegion dataRegion, final IndexMaintainer indexMaintainer,
+            final Region dataRegion, final IndexMaintainer indexMaintainer,
             final byte[][] viewConstants, final TupleProjector projector,
             final ImmutableBytesWritable ptr) {
         return getWrappedScanner(c, s, null, null, offset, scan, dataColumns, tupleProjector,
@@ -246,7 +246,7 @@ abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
             final RegionScanner s, final Set<KeyValueColumnExpression> arrayKVRefs,
             final Expression[] arrayFuncRefs, final int offset, final Scan scan,
             final ColumnReference[] dataColumns, final TupleProjector tupleProjector,
-            final HRegion dataRegion, final IndexMaintainer indexMaintainer,
+            final Region dataRegion, final IndexMaintainer indexMaintainer,
             final byte[][] viewConstants, final KeyValueSchema kvSchema,
             final ValueBitSet kvSchemaBitSet, final TupleProjector projector,
             final ImmutableBytesWritable ptr) {
@@ -257,7 +257,7 @@ abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
                 try {
                     return s.next(results);
                 } catch (Throwable t) {
-                    ServerUtil.throwIOException(c.getEnvironment().getRegion().getRegionNameAsString(), t);
+                    ServerUtil.throwIOException(c.getEnvironment().getRegion().getRegionInfo().getRegionNameAsString(), t);
                     return false; // impossible
                 }
             }
@@ -267,7 +267,7 @@ abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
                 try {
                     return s.next(result, scannerContext);
                 } catch (Throwable t) {
-                    ServerUtil.throwIOException(c.getEnvironment().getRegion().getRegionNameAsString(), t);
+                    ServerUtil.throwIOException(c.getEnvironment().getRegion().getRegionInfo().getRegionNameAsString(), t);
                     return false; // impossible
                 }
             }
@@ -319,7 +319,7 @@ abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
                     // There is a scanattribute set to retrieve the specific array element
                     return next;
                 } catch (Throwable t) {
-                    ServerUtil.throwIOException(c.getEnvironment().getRegion().getRegionNameAsString(), t);
+                    ServerUtil.throwIOException(c.getEnvironment().getRegion().getRegionInfo().getRegionNameAsString(), t);
                     return false; // impossible
                 }
             }
@@ -346,10 +346,10 @@ abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
                 }
                 // There is a scanattribute set to retrieve the specific array element
                 return next;
-            } catch (Throwable t) {
-                ServerUtil.throwIOException(c.getEnvironment().getRegion().getRegionNameAsString(), t);
+              } catch (Throwable t) {
+                ServerUtil.throwIOException(c.getEnvironment().getRegion().getRegionInfo().getRegionNameAsString(), t);
                 return false; // impossible
-            }
+              }
             }
 
             private void replaceArrayIndexElement(final Set<KeyValueColumnExpression> arrayKVRefs,

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ea622d5f/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupedAggregateRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupedAggregateRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupedAggregateRegionObserver.java
index 19a1663..d613688 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupedAggregateRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupedAggregateRegionObserver.java
@@ -45,7 +45,7 @@ import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 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.Region;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.WritableUtils;
@@ -112,8 +112,9 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
              * For local indexes, we need to set an offset on row key expressions to skip
              * the region start key.
              */
-            HRegion region = c.getEnvironment().getRegion();
-            offset = region.getStartKey().length != 0 ? region.getStartKey().length:region.getEndKey().length;
+            Region region = c.getEnvironment().getRegion();
+            offset = region.getRegionInfo().getStartKey().length != 0 ? region.getRegionInfo().getStartKey().length :
+                region.getRegionInfo().getEndKey().length;
             ScanUtil.setRowKeyOffset(scan, offset);
         }
 
@@ -128,7 +129,7 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
         byte[] localIndexBytes = scan.getAttribute(LOCAL_INDEX_BUILD);
         List<IndexMaintainer> indexMaintainers = localIndexBytes == null ? null : IndexMaintainer.deserialize(localIndexBytes);
         TupleProjector tupleProjector = null;
-        HRegion dataRegion = null;
+        Region dataRegion = null;
         byte[][] viewConstants = null;
         ColumnReference[] dataColumns = IndexUtil.deserializeDataTableColumnsToJoin(scan);
 
@@ -415,7 +416,7 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
                 logger.debug(LogUtil.addCustomAnnotations("Spillable groupby enabled: " + spillableEnabled, ScanUtil.getCustomAnnotations(scan)));
             }
 
-            HRegion region = c.getEnvironment().getRegion();
+            Region region = c.getEnvironment().getRegion();
             region.startRegionOperation();
             try {
                 synchronized (scanner) {
@@ -495,7 +496,7 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
                 // If we're calculating no aggregate functions, we can exit at the
                 // start of a new row. Otherwise, we have to wait until an agg
                 int countOffset = rowAggregators.length == 0 ? 1 : 0;
-                HRegion region = c.getEnvironment().getRegion();
+                Region region = c.getEnvironment().getRegion();
                 region.startRegionOperation();
                 try {
                     synchronized (scanner) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ea622d5f/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
index e613007..39a4956 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
@@ -99,8 +99,8 @@ import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
 import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.HRegion.RowLock;
+import org.apache.hadoop.hbase.regionserver.Region;
+import org.apache.hadoop.hbase.regionserver.Region.RowLock;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
@@ -404,7 +404,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         try {
             // TODO: check that key is within region.getStartKey() and region.getEndKey()
             // and return special code to force client to lookup region from meta.
-            HRegion region = env.getRegion();
+            Region region = env.getRegion();
             MetaDataMutationResult result = checkTableKeyInRegion(key, region);
             if (result != null) {
                 done.run(MetaDataMutationResult.toProto(result));
@@ -434,7 +434,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         }
     }
 
-    private PTable buildTable(byte[] key, ImmutableBytesPtr cacheKey, HRegion region,
+    private PTable buildTable(byte[] key, ImmutableBytesPtr cacheKey, Region region,
             long clientTimeStamp) throws IOException, SQLException {
         Scan scan = MetaDataUtil.newTableRowsScan(key, MIN_TABLE_TIMESTAMP, clientTimeStamp);
         RegionScanner scanner = region.getScanner(scan);
@@ -464,7 +464,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         }
     }
 
-    private List<PFunction> buildFunctions(List<byte[]> keys, HRegion region,
+    private List<PFunction> buildFunctions(List<byte[]> keys, Region region,
             long clientTimeStamp) throws IOException, SQLException {
         List<KeyRange> keyRanges = Lists.newArrayListWithExpectedSize(keys.size());
         for (byte[] key : keys) {
@@ -914,7 +914,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 className.getString(), jarPath == null ? null : jarPath.getString(), timeStamp);
     }
     
-    private PTable buildDeletedTable(byte[] key, ImmutableBytesPtr cacheKey, HRegion region,
+    private PTable buildDeletedTable(byte[] key, ImmutableBytesPtr cacheKey, Region region,
         long clientTimeStamp) throws IOException {
         if (clientTimeStamp == HConstants.LATEST_TIMESTAMP) {
             return null;
@@ -942,7 +942,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     }
 
     
-    private PFunction buildDeletedFunction(byte[] key, ImmutableBytesPtr cacheKey, HRegion region,
+    private PFunction buildDeletedFunction(byte[] key, ImmutableBytesPtr cacheKey, Region region,
         long clientTimeStamp) throws IOException {
         if (clientTimeStamp == HConstants.LATEST_TIMESTAMP) {
             return null;
@@ -989,7 +989,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     private PTable loadTable(RegionCoprocessorEnvironment env, byte[] key,
         ImmutableBytesPtr cacheKey, long clientTimeStamp, long asOfTimeStamp)
         throws IOException, SQLException {
-        HRegion region = env.getRegion();
+        Region region = env.getRegion();
         Cache<ImmutableBytesPtr,PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
         PTable table = (PTable)metaDataCache.getIfPresent(cacheKey);
         // We always cache the latest version - fault in if not in cache
@@ -1008,7 +1008,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     private PFunction loadFunction(RegionCoprocessorEnvironment env, byte[] key,
             ImmutableBytesPtr cacheKey, long clientTimeStamp, long asOfTimeStamp)
             throws IOException, SQLException {
-            HRegion region = env.getRegion();
+            Region region = env.getRegion();
             Cache<ImmutableBytesPtr,PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
             PFunction function = (PFunction)metaDataCache.getIfPresent(cacheKey);
             // We always cache the latest version - fault in if not in cache
@@ -1051,7 +1051,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                         schemaName, tableName);
             byte[] parentKey = parentTableName == null ? null : lockKey;
 
-            HRegion region = env.getRegion();
+            Region region = env.getRegion();
             MetaDataMutationResult result = checkTableKeyInRegion(lockKey, region);
             if (result != null) {
                 done.run(MetaDataMutationResult.toProto(result));
@@ -1115,7 +1115,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                         return;
                     }
                 }
-                // TODO: Switch this to HRegion#batchMutate when we want to support indexes on the
+                // TODO: Switch this to Region#batchMutate when we want to support indexes on the
                 // system
                 // table. Basically, we get all the locks that we don't already hold for all the
                 // tableMetadata rows. This ensures we don't have deadlock situations (ensuring
@@ -1125,7 +1125,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 // on the system table. This is an issue because of the way we manage batch mutation
                 // in the
                 // Indexer.
-                region.mutateRowsWithLocks(tableMetadata, Collections.<byte[]> emptySet());
+                region.mutateRowsWithLocks(tableMetadata, Collections.<byte[]> emptySet(), HConstants.NO_NONCE,
+                    HConstants.NO_NONCE);
 
                 // Invalidate the cache - the next getTable call will add it
                 // TODO: consider loading the table that was just created here, patching up the parent table, and updating the cache
@@ -1151,9 +1152,9 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     }
 
 
-    private static void acquireLock(HRegion region, byte[] key, List<RowLock> locks)
+    private static void acquireLock(Region region, byte[] key, List<RowLock> locks)
         throws IOException {
-        RowLock rowLock = region.getRowLock(key);
+        RowLock rowLock = region.getRowLock(key, true);
         if (rowLock == null) {
             throw new IOException("Failed to acquire lock on " + Bytes.toStringBinary(key));
         }
@@ -1167,7 +1168,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
      * TODO: should we pass a timestamp here?
      */
     @SuppressWarnings("deprecation")
-    private TableViewFinderResult findChildViews(HRegion region, byte[] tenantId, PTable table) throws IOException {
+    private TableViewFinderResult findChildViews(Region region, byte[] tenantId, PTable table) throws IOException {
         byte[] schemaName = table.getSchemaName().getBytes();
         byte[] tableName = table.getTableName().getBytes();
         boolean isMultiTenant = table.isMultiTenant();
@@ -1256,7 +1257,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     parentTableName == null ? lockKey : SchemaUtil.getTableKey(tenantIdBytes,
                         schemaName, tableName);
 
-            HRegion region = env.getRegion();
+            Region region = env.getRegion();
             MetaDataMutationResult result = checkTableKeyInRegion(key, region);
             if (result != null) {
                 done.run(MetaDataMutationResult.toProto(result));
@@ -1280,7 +1281,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 }
                 Cache<ImmutableBytesPtr,PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
                 // Commit the list of deletion.
-                region.mutateRowsWithLocks(tableMetadata, Collections.<byte[]> emptySet());
+                region.mutateRowsWithLocks(tableMetadata, Collections.<byte[]> emptySet(), HConstants.NO_NONCE,
+                    HConstants.NO_NONCE);
                 long currentTime = MetaDataUtil.getClientTimeStamp(tableMetadata);
                 for (ImmutableBytesPtr ckey : invalidateList) {
                     metaDataCache.put(ckey, newDeletedTableMarker(currentTime));
@@ -1309,7 +1311,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
 
         long clientTimeStamp = MetaDataUtil.getClientTimeStamp(rowsToDelete);
 
-        HRegion region = env.getRegion();
+        Region region = env.getRegion();
         ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(key);
 
         Cache<ImmutableBytesPtr,PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
@@ -1435,7 +1437,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
 
     private static interface ColumnMutator {
       MetaDataMutationResult updateMutation(PTable table, byte[][] rowKeyMetaData,
-          List<Mutation> tableMetadata, HRegion region,
+          List<Mutation> tableMetadata, Region region,
           List<ImmutableBytesPtr> invalidateList, List<RowLock> locks) throws IOException,
           SQLException;
     }
@@ -1449,7 +1451,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         byte[] tableName = rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
         try {
             byte[] key = SchemaUtil.getTableKey(tenantId, schemaName, tableName);
-            HRegion region = env.getRegion();
+            Region region = env.getRegion();
             MetaDataMutationResult result = checkTableKeyInRegion(key, region);
             if (result != null) {
                 return result;
@@ -1535,7 +1537,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     return result;
                 }
 
-                region.mutateRowsWithLocks(tableMetadata, Collections.<byte[]> emptySet());
+                region.mutateRowsWithLocks(tableMetadata, Collections.<byte[]> emptySet(), HConstants.NO_NONCE,
+                    HConstants.NO_NONCE);
                 // Invalidate from cache
                 for (ImmutableBytesPtr invalidateKey : invalidateList) {
                     metaDataCache.invalidate(invalidateKey);
@@ -1563,7 +1566,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             MetaDataMutationResult result = mutateColumn(tableMetaData, new ColumnMutator() {
                 @Override
                 public MetaDataMutationResult updateMutation(PTable table, byte[][] rowKeyMetaData,
-                        List<Mutation> tableMetaData, HRegion region,
+                        List<Mutation> tableMetaData, Region region,
                         List<ImmutableBytesPtr> invalidateList, List<RowLock> locks) {
                     byte[] tenantId = rowKeyMetaData[TENANT_ID_INDEX];
                     byte[] schemaName = rowKeyMetaData[SCHEMA_NAME_INDEX];
@@ -1647,14 +1650,14 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         // get the co-processor environment
         // TODO: check that key is within region.getStartKey() and region.getEndKey()
         // and return special code to force client to lookup region from meta.
-        HRegion region = env.getRegion();
+        Region region = env.getRegion();
         /*
          * Lock directly on key, though it may be an index table. This will just prevent a table
          * from getting rebuilt too often.
          */
         final boolean wasLocked = (rowLock != null);
         if (!wasLocked) {
-            rowLock = region.getRowLock(key);
+            rowLock = region.getRowLock(key, true);
             if (rowLock == null) {
                 throw new IOException("Failed to acquire lock on " + Bytes.toStringBinary(key));
             }
@@ -1689,7 +1692,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     private List<PFunction> doGetFunctions(List<byte[]> keys, long clientTimeStamp) throws IOException, SQLException {
         Cache<ImmutableBytesPtr, PMetaDataEntity> metaDataCache =
                 GlobalCache.getInstance(this.env).getMetaDataCache();
-        HRegion region = env.getRegion();
+        Region region = env.getRegion();
         Collections.sort(keys, new Comparator<byte[]>() {
             @Override
             public int compare(byte[] o1, byte[] o2) {
@@ -1700,11 +1703,11 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
          * Lock directly on key, though it may be an index table. This will just prevent a table
          * from getting rebuilt too often.
          */
-        List<RowLock> rowLocks = new ArrayList<HRegion.RowLock>(keys.size());;
+        List<RowLock> rowLocks = new ArrayList<Region.RowLock>(keys.size());;
         try {
-            rowLocks = new ArrayList<HRegion.RowLock>(keys.size());
+            rowLocks = new ArrayList<Region.RowLock>(keys.size());
             for (int i = 0; i < keys.size(); i++) {
-                HRegion.RowLock rowLock = region.getRowLock(keys.get(i));
+                Region.RowLock rowLock = region.getRowLock(keys.get(i), true);
                 if (rowLock == null) {
                     throw new IOException("Failed to acquire lock on "
                             + Bytes.toStringBinary(keys.get(i)));
@@ -1737,7 +1740,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             if(functionsAvailable.size() == numFunctions) return functionsAvailable;
             return null;
         } finally {
-            for (HRegion.RowLock lock : rowLocks) {
+            for (Region.RowLock lock : rowLocks) {
                 lock.release();
             }
             rowLocks.clear();
@@ -1756,7 +1759,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             MetaDataMutationResult result = mutateColumn(tableMetaData, new ColumnMutator() {
                 @Override
                 public MetaDataMutationResult updateMutation(PTable table, byte[][] rowKeyMetaData,
-                        List<Mutation> tableMetaData, HRegion region,
+                        List<Mutation> tableMetaData, Region region,
                         List<ImmutableBytesPtr> invalidateList, List<RowLock> locks)
                         throws IOException, SQLException {
                     byte[] tenantId = rowKeyMetaData[TENANT_ID_INDEX];
@@ -1904,7 +1907,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             schemaName = rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
             tableName = rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
             byte[] key = SchemaUtil.getTableKey(tenantId, schemaName, tableName);
-            HRegion region = env.getRegion();
+            Region region = env.getRegion();
             MetaDataMutationResult result = checkTableKeyInRegion(key, region);
             if (result != null) {
                 done.run(MetaDataMutationResult.toProto(result));
@@ -1928,7 +1931,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             }
             PIndexState newState =
                     PIndexState.fromSerializedValue(newKV.getValueArray()[newKV.getValueOffset()]);
-            RowLock rowLock = region.getRowLock(key);
+            RowLock rowLock = region.getRowLock(key, true);
             if (rowLock == null) {
                 throw new IOException("Failed to acquire lock on " + Bytes.toStringBinary(key));
             }
@@ -2019,7 +2022,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                         p.add(TABLE_FAMILY_BYTES, QueryConstants.EMPTY_COLUMN_BYTES, timeStamp, ByteUtil.EMPTY_BYTE_ARRAY);
                         tableMetadata.add(p);
                     }
-                    region.mutateRowsWithLocks(tableMetadata, Collections.<byte[]> emptySet());
+                    region.mutateRowsWithLocks(tableMetadata, Collections.<byte[]> emptySet(), HConstants.NO_NONCE,
+                        HConstants.NO_NONCE);
                     // Invalidate from cache
                     Cache<ImmutableBytesPtr,PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
                     metaDataCache.invalidate(cacheKey);
@@ -2044,9 +2048,9 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         }
     }
 
-    private static MetaDataMutationResult checkTableKeyInRegion(byte[] key, HRegion region) {
-        byte[] startKey = region.getStartKey();
-        byte[] endKey = region.getEndKey();
+    private static MetaDataMutationResult checkTableKeyInRegion(byte[] key, Region region) {
+        byte[] startKey = region.getRegionInfo().getStartKey();
+        byte[] endKey = region.getRegionInfo().getEndKey();
         if (Bytes.compareTo(startKey, key) <= 0
                 && (Bytes.compareTo(HConstants.LAST_ROW, endKey) == 0 || Bytes.compareTo(key,
                     endKey) < 0)) {
@@ -2056,9 +2060,9 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 EnvironmentEdgeManager.currentTimeMillis(), null);
     }
 
-    private static MetaDataMutationResult checkFunctionKeyInRegion(byte[] key, HRegion region) {
-        byte[] startKey = region.getStartKey();
-        byte[] endKey = region.getEndKey();
+    private static MetaDataMutationResult checkFunctionKeyInRegion(byte[] key, Region region) {
+        byte[] startKey = region.getRegionInfo().getStartKey();
+        byte[] endKey = region.getRegionInfo().getEndKey();
         if (Bytes.compareTo(startKey, key) <= 0
                 && (Bytes.compareTo(HConstants.LAST_ROW, endKey) == 0 || Bytes.compareTo(key,
                     endKey) < 0)) {
@@ -2135,7 +2139,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         byte[] tenantId = request.getTenantId().toByteArray();
         List<String> functionNames = new ArrayList<>(request.getFunctionNamesCount());
         try {
-            HRegion region = env.getRegion();
+            Region region = env.getRegion();
             List<ByteString> functionNamesList = request.getFunctionNamesList();
             List<Long> functionTimestampsList = request.getFunctionTimestampsList();
             List<byte[]> keys = new ArrayList<byte[]>(request.getFunctionNamesCount());
@@ -2189,7 +2193,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             byte[] tenantIdBytes = rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
             functionName = rowKeyMetaData[PhoenixDatabaseMetaData.FUNTION_NAME_INDEX];
             byte[] lockKey = SchemaUtil.getFunctionKey(tenantIdBytes, functionName);
-            HRegion region = env.getRegion();
+            Region region = env.getRegion();
             MetaDataMutationResult result = checkFunctionKeyInRegion(lockKey, region);
             if (result != null) {
                 done.run(MetaDataMutationResult.toProto(result));
@@ -2225,7 +2229,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 }
                 // Don't store function info for temporary functions.
                 if(!temporaryFunction) {
-                    region.mutateRowsWithLocks(functionMetaData, Collections.<byte[]> emptySet());
+                    region.mutateRowsWithLocks(functionMetaData, Collections.<byte[]> emptySet(), HConstants.NO_NONCE, HConstants.NO_NONCE);
                 }
 
                 // Invalidate the cache - the next getFunction call will add it
@@ -2259,7 +2263,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             byte[] tenantIdBytes = rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
             functionName = rowKeyMetaData[PhoenixDatabaseMetaData.FUNTION_NAME_INDEX];
             byte[] lockKey = SchemaUtil.getFunctionKey(tenantIdBytes, functionName);
-            HRegion region = env.getRegion();
+            Region region = env.getRegion();
             MetaDataMutationResult result = checkFunctionKeyInRegion(lockKey, region);
             if (result != null) {
                 done.run(MetaDataMutationResult.toProto(result));
@@ -2278,7 +2282,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     done.run(MetaDataMutationResult.toProto(result));
                     return;
                 }
-                region.mutateRowsWithLocks(functionMetaData, Collections.<byte[]> emptySet());
+                region.mutateRowsWithLocks(functionMetaData, Collections.<byte[]> emptySet(), HConstants.NO_NONCE, HConstants.NO_NONCE);
 
                 Cache<ImmutableBytesPtr,PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
                 long currentTime = MetaDataUtil.getClientTimeStamp(functionMetaData);
@@ -2322,7 +2326,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                             EnvironmentEdgeManager.currentTimeMillis(), null);
                 }
                 invalidateList.add(new FunctionBytesPtr(keys.get(0)));
-                HRegion region = env.getRegion();
+                Region region = env.getRegion();
                 Scan scan = MetaDataUtil.newTableRowsScan(keys.get(0), MIN_TABLE_TIMESTAMP, clientTimeStamp);
                 List<Cell> results = Lists.newArrayList();
                 try (RegionScanner scanner = region.getScanner(scan);) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ea622d5f/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ScanRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ScanRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ScanRegionObserver.java
index 77e124d..54c688a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ScanRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ScanRegionObserver.java
@@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 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.Region;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.phoenix.cache.GlobalCache;
@@ -176,8 +176,9 @@ public class ScanRegionObserver extends BaseScannerRegionObserver {
              * For local indexes, we need to set an offset on row key expressions to skip
              * the region start key.
              */
-            HRegion region = c.getEnvironment().getRegion();
-            offset = region.getStartKey().length != 0 ? region.getStartKey().length:region.getEndKey().length;
+            Region region = c.getEnvironment().getRegion();
+            offset = region.getRegionInfo().getStartKey().length != 0 ? region.getRegionInfo().getStartKey().length :
+                region.getRegionInfo().getEndKey().length;
             ScanUtil.setRowKeyOffset(scan, offset);
         }
 
@@ -187,7 +188,7 @@ public class ScanRegionObserver extends BaseScannerRegionObserver {
         Expression[] arrayFuncRefs = deserializeArrayPostionalExpressionInfoFromScan(
                 scan, innerScanner, arrayKVRefs);
         TupleProjector tupleProjector = null;
-        HRegion dataRegion = null;
+        Region dataRegion = null;
         IndexMaintainer indexMaintainer = null;
         byte[][] viewConstants = null;
         ColumnReference[] dataColumns = IndexUtil.deserializeDataTableColumnsToJoin(scan);
@@ -231,7 +232,7 @@ public class ScanRegionObserver extends BaseScannerRegionObserver {
         TenantCache tenantCache = GlobalCache.getTenantCache(c.getEnvironment(), tenantId);
         long estSize = iterator.getEstimatedByteSize();
         final MemoryChunk chunk = tenantCache.getMemoryManager().allocate(estSize);
-        final HRegion region = c.getEnvironment().getRegion();
+        final Region region = c.getEnvironment().getRegion();
         region.startRegionOperation();
         try {
             // Once we return from the first call to next, we've run through and cached
@@ -241,7 +242,7 @@ public class ScanRegionObserver extends BaseScannerRegionObserver {
             long actualSize = iterator.getByteSize();
             chunk.resize(actualSize);
         } catch (Throwable t) {
-            ServerUtil.throwIOException(region.getRegionNameAsString(), t);
+            ServerUtil.throwIOException(region.getRegionInfo().getRegionNameAsString(), t);
             return null;
         } finally {
             region.closeRegionOperation();
@@ -273,7 +274,7 @@ public class ScanRegionObserver extends BaseScannerRegionObserver {
                     tuple = iterator.next();
                     return !isFilterDone();
                 } catch (Throwable t) {
-                    ServerUtil.throwIOException(region.getRegionNameAsString(), t);
+                    ServerUtil.throwIOException(region.getRegionInfo().getRegionNameAsString(), t);
                     return false;
                 }
             }
@@ -288,7 +289,7 @@ public class ScanRegionObserver extends BaseScannerRegionObserver {
                             iterator.close();
                         }
                     } catch (SQLException e) {
-                        ServerUtil.throwIOException(region.getRegionNameAsString(), e);
+                        ServerUtil.throwIOException(region.getRegionInfo().getRegionNameAsString(), e);
                     } finally {
                         chunk.close();
                     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ea622d5f/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/SequenceRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/SequenceRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/SequenceRegionObserver.java
index 7953933..9b5f040 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/SequenceRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/SequenceRegionObserver.java
@@ -38,8 +38,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.io.TimeRange;
-import org.apache.hadoop.hbase.regionserver.HRegion;
-import org.apache.hadoop.hbase.regionserver.HRegion.RowLock;
+import org.apache.hadoop.hbase.regionserver.Region;
+import org.apache.hadoop.hbase.regionserver.Region.RowLock;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.util.EnvironmentEdgeManager;
 import org.apache.phoenix.exception.SQLExceptionCode;
@@ -88,9 +88,9 @@ public class SequenceRegionObserver extends BaseRegionObserver {
                         QueryConstants.EMPTY_COLUMN_BYTES, timestamp, errorCodeBuf)));
     }
     
-    private static void acquireLock(HRegion region, byte[] key, List<RowLock> locks)
+    private static void acquireLock(Region region, byte[] key, List<RowLock> locks)
         throws IOException {
-        RowLock rowLock = region.getRowLock(key);
+        RowLock rowLock = region.getRowLock(key, true);
         if (rowLock == null) {
             throw new IOException("Failed to acquire lock on " + Bytes.toStringBinary(key));
         }
@@ -114,7 +114,7 @@ public class SequenceRegionObserver extends BaseRegionObserver {
         // We need to set this to prevent region.increment from being called
         e.bypass();
         e.complete();
-        HRegion region = env.getRegion();
+        Region region = env.getRegion();
         byte[] row = increment.getRow();
         List<RowLock> locks = Lists.newArrayList();
         TimeRange tr = increment.getTimeRange();
@@ -251,7 +251,7 @@ public class SequenceRegionObserver extends BaseRegionObserver {
                 }
                 // update the KeyValues on the server
                 Mutation[] mutations = new Mutation[]{put};
-                region.batchMutate(mutations);
+                region.batchMutate(mutations, HConstants.NO_NONCE, HConstants.NO_NONCE);
                 // return a Result with the updated KeyValues
                 return Result.create(cells);
             } finally {
@@ -345,7 +345,7 @@ public class SequenceRegionObserver extends BaseRegionObserver {
         // We need to set this to prevent region.append from being called
         e.bypass();
         e.complete();
-        HRegion region = env.getRegion();
+        Region region = env.getRegion();
         byte[] row = append.getRow();
         List<RowLock> locks = Lists.newArrayList();
         region.startRegionOperation();
@@ -400,7 +400,7 @@ public class SequenceRegionObserver extends BaseRegionObserver {
                     }
                 }
                 Mutation[] mutations = new Mutation[]{m};
-                region.batchMutate(mutations);
+                region.batchMutate(mutations, HConstants.NO_NONCE, HConstants.NO_NONCE);
                 long serverTimestamp = MetaDataUtil.getClientTimeStamp(m);
                 // Return result with single KeyValue. The only piece of information
                 // the client cares about is the timestamp, which is the timestamp of

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ea622d5f/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
index 2d6d98a..d5cc486 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
@@ -48,8 +48,8 @@ import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 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.RegionScanner;
 import org.apache.hadoop.hbase.regionserver.ScanType;
 import org.apache.hadoop.hbase.regionserver.Store;
@@ -125,7 +125,7 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver{
         this.kvBuilder = GenericKeyValueBuilder.INSTANCE;
     }
 
-    private static void commitBatch(HRegion region, List<Mutation> mutations, byte[] indexUUID) throws IOException {
+    private static void commitBatch(Region region, List<Mutation> mutations, byte[] indexUUID) throws IOException {
       if (indexUUID != null) {
           for (Mutation m : mutations) {
               m.setAttribute(PhoenixIndexCodec.INDEX_UUID, indexUUID);
@@ -133,7 +133,7 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver{
       }
       Mutation[] mutationArray = new Mutation[mutations.size()];
       // TODO: should we use the one that is all or none?
-      region.batchMutate(mutations.toArray(mutationArray));
+      region.batchMutate(mutations.toArray(mutationArray), HConstants.NO_NONCE, HConstants.NO_NONCE);
     }
 
     public static void serializeIntoScan(Scan scan) {
@@ -158,7 +158,7 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver{
     @Override
     protected RegionScanner doPostScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c, final Scan scan, final RegionScanner s) throws IOException {
         int offset = 0;
-        HRegion region = c.getEnvironment().getRegion();
+        Region region = c.getEnvironment().getRegion();
         long ts = scan.getTimeRange().getMax();
         StatisticsCollector stats = null;
         if(ScanUtil.isAnalyzeTable(scan)) {
@@ -172,7 +172,8 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver{
              * For local indexes, we need to set an offset on row key expressions to skip
              * the region start key.
              */
-            offset = region.getStartKey().length != 0 ? region.getStartKey().length:region.getEndKey().length;
+            offset = region.getRegionInfo().getStartKey().length != 0 ? region.getRegionInfo().getStartKey().length :
+                region.getRegionInfo().getEndKey().length;
             ScanUtil.setRowKeyOffset(scan, offset);
         }
 
@@ -212,7 +213,7 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver{
             ptr = new ImmutableBytesWritable();
         }
         TupleProjector tupleProjector = null;
-        HRegion dataRegion = null;
+        Region dataRegion = null;
         byte[][] viewConstants = null;
         ColumnReference[] dataColumns = IndexUtil.deserializeDataTableColumnsToJoin(scan);
         boolean localIndexScan = ScanUtil.isLocalIndex(scan);
@@ -279,8 +280,8 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver{
                                                 results);
                                         Put put = maintainer.buildUpdateMutation(kvBuilder,
                                             valueGetter, ptr, ts,
-                                            c.getEnvironment().getRegion().getStartKey(),
-                                            c.getEnvironment().getRegion().getEndKey());
+                                            c.getEnvironment().getRegion().getRegionInfo().getStartKey(),
+                                            c.getEnvironment().getRegion().getRegionInfo().getEndKey());
                                         indexMutations.add(put);
                                     }
                                 }
@@ -391,7 +392,7 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver{
                         } catch (ConstraintViolationException e) {
                             // Log and ignore in count
                             logger.error(LogUtil.addCustomAnnotations("Failed to create row in " +
-                                region.getRegionNameAsString() + " with values " +
+                                region.getRegionInfo().getRegionNameAsString() + " with values " +
                                 SchemaUtil.toString(values),
                                 ScanUtil.getCustomAnnotations(scan)), e);
                             continue;
@@ -479,9 +480,9 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver{
     }
 
     private void commitIndexMutations(final ObserverContext<RegionCoprocessorEnvironment> c,
-            HRegion region, List<Mutation> indexMutations) throws IOException {
+            Region region, List<Mutation> indexMutations) throws IOException {
         // Get indexRegion corresponding to data region
-        HRegion indexRegion = IndexUtil.getIndexRegion(c.getEnvironment());
+        Region indexRegion = IndexUtil.getIndexRegion(c.getEnvironment());
         if (indexRegion != null) {
             commitBatch(indexRegion, indexMutations, null);
         } else {
@@ -493,7 +494,7 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver{
                 table = c.getEnvironment().getTable(indexTable);
                 table.batch(indexMutations);
             } catch (InterruptedException ie) {
-                ServerUtil.throwIOException(c.getEnvironment().getRegion().getRegionNameAsString(),
+                ServerUtil.throwIOException(c.getEnvironment().getRegion().getRegionInfo().getRegionNameAsString(),
                     ie);
             } finally {
                 if (table != null) table.close();
@@ -534,9 +535,9 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver{
 
 
     @Override
-    public void postSplit(ObserverContext<RegionCoprocessorEnvironment> e, HRegion l, HRegion r)
+    public void postSplit(ObserverContext<RegionCoprocessorEnvironment> e, Region l, Region r)
             throws IOException {
-        HRegion region = e.getEnvironment().getRegion();
+        Region region = e.getEnvironment().getRegion();
         TableName table = region.getRegionInfo().getTable();
         StatisticsCollector stats = null;
         try {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ea622d5f/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/data/LocalTable.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/data/LocalTable.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/data/LocalTable.java
index 71cc1d6..549fe8c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/data/LocalTable.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/data/LocalTable.java
@@ -24,12 +24,11 @@ import java.util.Collections;
 import java.util.List;
 
 import org.apache.hadoop.hbase.Cell;
-import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Result;
 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.phoenix.hbase.index.covered.update.ColumnReference;
@@ -60,7 +59,7 @@ public class LocalTable implements LocalHBaseState {
     Scan s = IndexManagementUtil.newLocalStateScan(Collections.singletonList(columns));
     s.setStartRow(row);
     s.setStopRow(row);
-    HRegion region = this.env.getRegion();
+    Region region = this.env.getRegion();
     RegionScanner scanner = region.getScanner(s);
     List<Cell> kvs = new ArrayList<Cell>(1);
     boolean more = scanner.next(kvs);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ea622d5f/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/ParallelWriterIndexCommitter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/ParallelWriterIndexCommitter.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/ParallelWriterIndexCommitter.java
index f72dec0..56bf637 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/ParallelWriterIndexCommitter.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/ParallelWriterIndexCommitter.java
@@ -21,11 +21,12 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.phoenix.hbase.index.exception.SingleIndexWriteFailureException;
 import org.apache.phoenix.hbase.index.parallel.EarlyExitFailure;
 import org.apache.phoenix.hbase.index.parallel.QuickFailingTaskRunner;
@@ -150,10 +151,11 @@ public class ParallelWriterIndexCommitter implements IndexCommitter {
                         // as well.
                         try {
                             if (tableReference.getTableName().startsWith(MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX)) {
-                                HRegion indexRegion = IndexUtil.getIndexRegion(env);
+                                Region indexRegion = IndexUtil.getIndexRegion(env);
                                 if (indexRegion != null) {
                                     throwFailureIfDone();
-                                    indexRegion.batchMutate(mutations.toArray(new Mutation[mutations.size()]));
+                                    indexRegion.batchMutate(mutations.toArray(new Mutation[mutations.size()]),
+                                        HConstants.NO_NONCE, HConstants.NO_NONCE);
                                     return null;
                                 }
                             }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ea622d5f/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/recovery/PerRegionIndexWriteCache.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/recovery/PerRegionIndexWriteCache.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/recovery/PerRegionIndexWriteCache.java
index 4d5f667..26da2d5 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/recovery/PerRegionIndexWriteCache.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/recovery/PerRegionIndexWriteCache.java
@@ -22,7 +22,7 @@ import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.Region;
 
 import com.google.common.collect.ArrayListMultimap;
 import com.google.common.collect.Multimap;
@@ -32,8 +32,8 @@ import org.apache.phoenix.hbase.index.table.HTableInterfaceReference;
 
 public class PerRegionIndexWriteCache {
 
-  private Map<HRegion, Multimap<HTableInterfaceReference, Mutation>> cache =
-      new HashMap<HRegion, Multimap<HTableInterfaceReference, Mutation>>();
+  private Map<Region, Multimap<HTableInterfaceReference, Mutation>> cache =
+      new HashMap<Region, Multimap<HTableInterfaceReference, Mutation>>();
 
 
   /**
@@ -43,7 +43,7 @@ public class PerRegionIndexWriteCache {
    * @return Get the edits for the given region. Returns <tt>null</tt> if there are no pending edits
    *         for the region
    */
-  public Multimap<HTableInterfaceReference, Mutation> getEdits(HRegion region) {
+  public Multimap<HTableInterfaceReference, Mutation> getEdits(Region region) {
     return cache.remove(region);
   }
 
@@ -52,7 +52,7 @@ public class PerRegionIndexWriteCache {
    * @param table
    * @param collection
    */
-  public void addEdits(HRegion region, HTableInterfaceReference table,
+  public void addEdits(Region region, HTableInterfaceReference table,
       Collection<Mutation> collection) {
     Multimap<HTableInterfaceReference, Mutation> edits = cache.get(region);
     if (edits == null) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ea622d5f/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/recovery/StoreFailuresInCachePolicy.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/recovery/StoreFailuresInCachePolicy.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/recovery/StoreFailuresInCachePolicy.java
index f36affb..189f970 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/recovery/StoreFailuresInCachePolicy.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/recovery/StoreFailuresInCachePolicy.java
@@ -23,7 +23,7 @@ import java.util.List;
 import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.Region;
 
 import com.google.common.collect.Multimap;
 import org.apache.phoenix.hbase.index.exception.MultiIndexWriteFailureException;
@@ -41,7 +41,7 @@ public class StoreFailuresInCachePolicy implements IndexFailurePolicy {
 
   private KillServerOnFailurePolicy delegate;
   private PerRegionIndexWriteCache cache;
-  private HRegion region;
+  private Region region;
 
   /**
    * @param failedIndexEdits cache to update when we find a failure

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ea622d5f/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/recovery/TrackingParallelWriterIndexCommitter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/recovery/TrackingParallelWriterIndexCommitter.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/recovery/TrackingParallelWriterIndexCommitter.java
index 9171b53..b1b2656 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/recovery/TrackingParallelWriterIndexCommitter.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/recovery/TrackingParallelWriterIndexCommitter.java
@@ -23,11 +23,12 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Abortable;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.phoenix.hbase.index.CapturingAbortable;
 import org.apache.phoenix.hbase.index.exception.MultiIndexWriteFailureException;
 import org.apache.phoenix.hbase.index.exception.SingleIndexWriteFailureException;
@@ -154,10 +155,11 @@ public class TrackingParallelWriterIndexCommitter implements IndexCommitter {
                             // index is pretty hacky. If we're going to keep this, we should revisit that
                             // as well.
                             if (tableReference.getTableName().startsWith(MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX)) {
-                                HRegion indexRegion = IndexUtil.getIndexRegion(env);
+                                Region indexRegion = IndexUtil.getIndexRegion(env);
                                 if (indexRegion != null) {
                                     throwFailureIfDone();
-                                    indexRegion.batchMutate(mutations.toArray(new Mutation[mutations.size()]));
+                                    indexRegion.batchMutate(mutations.toArray(new Mutation[mutations.size()]),
+                                        HConstants.NO_NONCE, HConstants.NO_NONCE);
                                     return Boolean.TRUE;
                                 }
                             }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ea622d5f/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexBuilder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexBuilder.java b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexBuilder.java
index b5e6a63..7a45e21 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexBuilder.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexBuilder.java
@@ -28,8 +28,8 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
+import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.phoenix.compile.ScanRanges;
 import org.apache.phoenix.hbase.index.covered.CoveredColumnsIndexBuilder;
@@ -73,7 +73,7 @@ public class PhoenixIndexBuilder extends CoveredColumnsIndexBuilder {
         ScanRanges scanRanges = ScanRanges.create(SchemaUtil.VAR_BINARY_SCHEMA, Collections.singletonList(keys), ScanUtil.SINGLE_COLUMN_SLOT_SPAN);
         scanRanges.initializeScan(scan);
         scan.setFilter(scanRanges.getSkipScanFilter());
-        HRegion region = this.env.getRegion();
+        Region region = this.env.getRegion();
         RegionScanner scanner = region.getScanner(scan);
         // Run through the scanner using internal nextRaw method
         region.startRegionOperation();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ea622d5f/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexCodec.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexCodec.java b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexCodec.java
index 99e26d1..222aefb 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexCodec.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexCodec.java
@@ -24,9 +24,7 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Pair;
@@ -166,14 +164,14 @@ public class PhoenixIndexCodec extends BaseIndexCodec {
             Mutation mutation = null;
             if (upsert) {
                 mutation =
-                        maintainer.buildUpdateMutation(kvBuilder, valueGetter, ptr, state
-                                .getCurrentTimestamp(), env.getRegion().getStartKey(), env
-                                .getRegion().getEndKey());
+                        maintainer.buildUpdateMutation(kvBuilder, valueGetter, ptr, state.getCurrentTimestamp(),
+                            env.getRegion().getRegionInfo().getStartKey(),
+                            env.getRegion().getRegionInfo().getEndKey());
             } else {
                 mutation =
-                        maintainer.buildDeleteMutation(kvBuilder, valueGetter, ptr, state
-                                .getPendingUpdate(), state.getCurrentTimestamp(), env.getRegion()
-                                .getStartKey(), env.getRegion().getEndKey());
+                        maintainer.buildDeleteMutation(kvBuilder, valueGetter, ptr, state.getPendingUpdate(),
+                            state.getCurrentTimestamp(), env.getRegion().getRegionInfo().getStartKey(),
+                            env.getRegion().getRegionInfo().getEndKey());
             }
             indexUpdate.setUpdate(mutation);
             if (scanner != null) {


[17/31] phoenix git commit: PHOENIX-1976 Exit gracefully if addShutdownHook fails.

Posted by nd...@apache.org.
PHOENIX-1976 Exit gracefully if addShutdownHook fails.

If the JVM is already in the process of shutting down,
we don't need to add the shutdown hook for the PhoenixDriver
instance. Additionally, we shouldn't advertise this instance
either since we're going down.


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

Branch: refs/heads/4.x-HBase-1.1
Commit: f2be9138359b078fd3e285f3fd441de711789962
Parents: dc46b14
Author: Josh Elser <jo...@gmail.com>
Authored: Thu May 14 17:40:46 2015 -0400
Committer: Nick Dimiduk <nd...@apache.org>
Committed: Mon Jun 1 12:02:28 2015 -0700

----------------------------------------------------------------------
 .../org/apache/phoenix/jdbc/PhoenixDriver.java  | 46 ++++++++++++++------
 1 file changed, 32 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/f2be9138/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDriver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDriver.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDriver.java
index 6360d06..cfabe82 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDriver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDriver.java
@@ -60,25 +60,43 @@ public final class PhoenixDriver extends PhoenixEmbeddedDriver {
     private static volatile String driverShutdownMsg;
     static {
         try {
-            DriverManager.registerDriver( INSTANCE = new PhoenixDriver() );
-            // Add shutdown hook to release any resources that were never closed
-            // In theory not necessary, but it won't hurt anything
-            Runtime.getRuntime().addShutdownHook(new Thread() {
-                @Override
-                public void run() {
-                    try {
-                        INSTANCE.close();
-                    } catch (SQLException e) {
-                        logger.warn("Unable to close PhoenixDriver on shutdown", e);
-                    } finally {
-                        driverShutdownMsg = "Phoenix driver closed because server is shutting down";
+            INSTANCE = new PhoenixDriver();
+            try {
+                // Add shutdown hook to release any resources that were never closed
+                // In theory not necessary, but it won't hurt anything
+                Runtime.getRuntime().addShutdownHook(new Thread() {
+                    @Override
+                    public void run() {
+                        closeInstance(INSTANCE);
                     }
-                }
-            });
+                });
+
+                // Only register the driver when we successfully register the shutdown hook
+                // Don't want to register it if we're already in the process of going down.
+                DriverManager.registerDriver( INSTANCE );
+            } catch (IllegalStateException e) {
+                logger.warn("Failed to register PhoenixDriver shutdown hook as the JVM is already shutting down");
+
+                // Close the instance now because we don't have the shutdown hook
+                closeInstance(INSTANCE);
+
+                throw e;
+            }
         } catch (SQLException e) {
             throw new IllegalStateException("Unable to register " + PhoenixDriver.class.getName() + ": "+ e.getMessage());
         }
     }
+
+    private static void closeInstance(PhoenixDriver instance) {
+        try {
+            instance.close();
+        } catch (SQLException e) {
+            logger.warn("Unable to close PhoenixDriver on shutdown", e);
+        } finally {
+            driverShutdownMsg = "Phoenix driver closed because server is shutting down";
+        }
+    }
+
     // One entry per cluster here
     private final ConcurrentMap<ConnectionInfo,ConnectionQueryServices> connectionQueryServicesMap = new ConcurrentHashMap<ConnectionInfo,ConnectionQueryServices>(3);
 


[10/31] phoenix git commit: PHOENIX-2013 Apply PHOENIX-1995 to runnable uberjar as well

Posted by nd...@apache.org.
PHOENIX-2013 Apply PHOENIX-1995 to runnable uberjar as well


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

Branch: refs/heads/4.x-HBase-1.1
Commit: 0e0b4ddb4d130b38c7aa28d2e31b0a9552087256
Parents: 1a2f2dc
Author: Nick Dimiduk <nd...@apache.org>
Authored: Wed May 27 11:27:04 2015 -0700
Committer: Nick Dimiduk <nd...@apache.org>
Committed: Wed May 27 13:20:32 2015 -0700

----------------------------------------------------------------------
 phoenix-server/src/build/query-server-runnable.xml | 9 +++++++++
 1 file changed, 9 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/0e0b4ddb/phoenix-server/src/build/query-server-runnable.xml
----------------------------------------------------------------------
diff --git a/phoenix-server/src/build/query-server-runnable.xml b/phoenix-server/src/build/query-server-runnable.xml
index e2a3dc4..ef22b14 100644
--- a/phoenix-server/src/build/query-server-runnable.xml
+++ b/phoenix-server/src/build/query-server-runnable.xml
@@ -28,6 +28,15 @@
     <format>jar</format>
   </formats>
   <includeBaseDirectory>false</includeBaseDirectory>
+  <containerDescriptorHandlers>
+    <containerDescriptorHandler>
+      <!--
+          aggregate SPI's so that things like HDFS FileSystem works in uberjar
+          http://docs.oracle.com/javase/tutorial/sound/SPI-intro.html
+      -->
+      <handlerName>metaInf-services</handlerName>
+    </containerDescriptorHandler>
+  </containerDescriptorHandlers>
   <dependencySets>
     <dependencySet>
       <outputDirectory>/</outputDirectory>


[07/31] phoenix git commit: PHOENIX-2005 Connection utilities omit zk client port, parent znode

Posted by nd...@apache.org.
PHOENIX-2005 Connection utilities omit zk client port, parent znode


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

Branch: refs/heads/4.x-HBase-1.1
Commit: c6b37b979da1b514bcb9257c7e095e39b0c2c215
Parents: 3cdc323
Author: Nick Dimiduk <nd...@apache.org>
Authored: Tue May 26 11:11:48 2015 -0700
Committer: Nick Dimiduk <nd...@apache.org>
Committed: Tue May 26 13:27:03 2015 -0700

----------------------------------------------------------------------
 .../phoenix/jdbc/PhoenixEmbeddedDriver.java     | 28 ++++--
 .../phoenix/mapreduce/CsvBulkLoadTool.java      | 93 ++++++++++----------
 .../phoenix/mapreduce/CsvToKeyValueMapper.java  | 26 +-----
 .../query/ConnectionQueryServicesImpl.java      |  4 +-
 .../java/org/apache/phoenix/util/QueryUtil.java | 45 ++++++++--
 .../phoenix/jdbc/PhoenixEmbeddedDriverTest.java | 14 ++-
 .../phoenix/mapreduce/CsvBulkLoadToolTest.java  | 11 ---
 .../mapreduce/CsvToKeyValueMapperTest.java      | 15 ----
 .../org/apache/phoenix/util/QueryUtilTest.java  | 33 ++++---
 9 files changed, 139 insertions(+), 130 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/c6b37b97/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixEmbeddedDriver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixEmbeddedDriver.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixEmbeddedDriver.java
index 9e95667..2451603 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixEmbeddedDriver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixEmbeddedDriver.java
@@ -31,6 +31,7 @@ import java.util.logging.Logger;
 
 import javax.annotation.concurrent.Immutable;
 
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.phoenix.coprocessor.MetaDataProtocol;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
@@ -174,10 +175,10 @@ public abstract class PhoenixEmbeddedDriver implements Driver, org.apache.phoeni
     }
     
     /**
-     * 
+     *
      * Class to encapsulate connection info for HBase
      *
-     * 
+     *
      * @since 0.1.1
      */
     public static class ConnectionInfo {
@@ -204,12 +205,18 @@ public abstract class PhoenixEmbeddedDriver implements Driver, org.apache.phoeni
             return false;
         }
         
-        protected static ConnectionInfo create(String url) throws SQLException {
-            StringTokenizer tokenizer = new StringTokenizer(url == null ? "" : url.substring(PhoenixRuntime.JDBC_PROTOCOL.length()),DELIMITERS, true);
+        public static ConnectionInfo create(String url) throws SQLException {
+            url = url == null ? "" : url;
+            url = url.startsWith(PhoenixRuntime.JDBC_PROTOCOL)
+                    ? url.substring(PhoenixRuntime.JDBC_PROTOCOL.length())
+                    : url;
+            StringTokenizer tokenizer = new StringTokenizer(url, DELIMITERS, true);
             int nTokens = 0;
             String[] tokens = new String[5];
             String token = null;
-            while (tokenizer.hasMoreTokens() && !(token=tokenizer.nextToken()).equals(TERMINATOR) && tokenizer.hasMoreTokens() && nTokens < tokens.length) {
+            while (tokenizer.hasMoreTokens() &&
+                    !(token=tokenizer.nextToken()).equals(TERMINATOR) &&
+                    tokenizer.hasMoreTokens() && nTokens < tokens.length) {
                 token = tokenizer.nextToken();
                 // This would mean we have an empty string for a token which is illegal
                 if (DELIMITERS.contains(token)) {
@@ -316,8 +323,7 @@ public abstract class PhoenixEmbeddedDriver implements Driver, org.apache.phoeni
         private final String principal;
         private final String keytab;
         
-        // used for testing
-        ConnectionInfo(String zookeeperQuorum, Integer port, String rootNode, String principal, String keytab) {
+        public ConnectionInfo(String zookeeperQuorum, Integer port, String rootNode, String principal, String keytab) {
             this.zookeeperQuorum = zookeeperQuorum;
             this.port = port;
             this.rootNode = rootNode;
@@ -326,8 +332,7 @@ public abstract class PhoenixEmbeddedDriver implements Driver, org.apache.phoeni
             this.keytab = keytab;
         }
         
-        // used for testing
-        ConnectionInfo(String zookeeperQuorum, Integer port, String rootNode) {
+        public ConnectionInfo(String zookeeperQuorum, Integer port, String rootNode) {
         	this(zookeeperQuorum, port, rootNode, null, null);
         }
 
@@ -417,6 +422,11 @@ public abstract class PhoenixEmbeddedDriver implements Driver, org.apache.phoeni
 					+ (principal == null ? "" : ":" + principal)
 					+ (keytab == null ? "" : ":" + keytab);
 		}
+
+        public String toUrl() {
+            return PhoenixRuntime.JDBC_PROTOCOL + PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR
+                    + toString();
+        }
     }
 
     public static boolean isTestUrl(String url) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c6b37b97/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/CsvBulkLoadTool.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/CsvBulkLoadTool.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/CsvBulkLoadTool.java
index 31f8b42..7afde98 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/CsvBulkLoadTool.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/CsvBulkLoadTool.java
@@ -18,11 +18,11 @@
 package org.apache.phoenix.mapreduce;
 
 import java.sql.Connection;
-import java.sql.DriverManager;
 import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
@@ -41,7 +41,6 @@ import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
@@ -56,8 +55,8 @@ import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.jdbc.PhoenixDriver;
 import org.apache.phoenix.job.JobManager;
-import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
@@ -67,6 +66,7 @@ import org.apache.phoenix.util.CSVCommonsLoader;
 import org.apache.phoenix.util.ColumnInfo;
 import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.StringUtil;
 import org.slf4j.Logger;
@@ -84,7 +84,7 @@ public class CsvBulkLoadTool extends Configured implements Tool {
 
     private static final Logger LOG = LoggerFactory.getLogger(CsvBulkLoadTool.class);
 
-    static final Option ZK_QUORUM_OPT = new Option("z", "zookeeper", true, "Zookeeper quorum to connect to (optional)");
+    static final Option ZK_QUORUM_OPT = new Option("z", "zookeeper", true, "Supply zookeeper connection details (optional)");
     static final Option INPUT_PATH_OPT = new Option("i", "input", true, "Input CSV path (mandatory)");
     static final Option OUTPUT_PATH_OPT = new Option("o", "output", true, "Output path for temporary HFiles (optional)");
     static final Option SCHEMA_NAME_OPT = new Option("s", "schema", true, "Phoenix schema name (optional)");
@@ -184,35 +184,48 @@ public class CsvBulkLoadTool extends Configured implements Tool {
         } catch (IllegalStateException e) {
             printHelpAndExit(e.getMessage(), getOptions());
         }
-        Class.forName(DriverManager.class.getName());
-        Connection conn = DriverManager.getConnection(
-                getJdbcUrl(cmdLine.getOptionValue(ZK_QUORUM_OPT.getOpt())));
-        
-        return loadData(conf, cmdLine, conn);
+        return loadData(conf, cmdLine);
     }
 
-	private int loadData(Configuration conf, CommandLine cmdLine,
-			Connection conn) throws SQLException, InterruptedException,
-			ExecutionException {
-		    String tableName = cmdLine.getOptionValue(TABLE_NAME_OPT.getOpt());
+	private int loadData(Configuration conf, CommandLine cmdLine) throws SQLException,
+            InterruptedException, ExecutionException, ClassNotFoundException {
+        String tableName = cmdLine.getOptionValue(TABLE_NAME_OPT.getOpt());
         String schemaName = cmdLine.getOptionValue(SCHEMA_NAME_OPT.getOpt());
         String indexTableName = cmdLine.getOptionValue(INDEX_TABLE_NAME_OPT.getOpt());
         String qualifiedTableName = getQualifiedTableName(schemaName, tableName);
-        String qualifedIndexTableName = null;
-        if(indexTableName != null){
-        	qualifedIndexTableName = getQualifiedTableName(schemaName, indexTableName);
+        String qualifiedIndexTableName = null;
+        if (indexTableName != null){
+        	qualifiedIndexTableName = getQualifiedTableName(schemaName, indexTableName);
+        }
+
+        if (cmdLine.hasOption(ZK_QUORUM_OPT.getOpt())) {
+            // ZK_QUORUM_OPT is optional, but if it's there, use it for both the conn and the job.
+            String zkQuorum = cmdLine.getOptionValue(ZK_QUORUM_OPT.getOpt());
+            PhoenixDriver.ConnectionInfo info = PhoenixDriver.ConnectionInfo.create(zkQuorum);
+            LOG.info("Configuring HBase connection to {}", info);
+            for (Map.Entry<String,String> entry : info.asProps()) {
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("Setting {} = {}", entry.getKey(), entry.getValue());
+                }
+                conf.set(entry.getKey(), entry.getValue());
+            }
+        }
+
+        final Connection conn = QueryUtil.getConnection(conf);
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Reading columns from {} :: {}", ((PhoenixConnection) conn).getURL(),
+                    qualifiedTableName);
         }
         List<ColumnInfo> importColumns = buildImportColumns(conn, cmdLine, qualifiedTableName);
         configureOptions(cmdLine, importColumns, conf);
-
         try {
             validateTable(conn, schemaName, tableName);
         } finally {
             conn.close();
         }
 
-        Path inputPath = new Path(cmdLine.getOptionValue(INPUT_PATH_OPT.getOpt()));
-        Path outputPath = null;
+        final Path inputPath = new Path(cmdLine.getOptionValue(INPUT_PATH_OPT.getOpt()));
+        final Path outputPath;
         if (cmdLine.hasOption(OUTPUT_PATH_OPT.getOpt())) {
             outputPath = new Path(cmdLine.getOptionValue(OUTPUT_PATH_OPT.getOpt()));
         } else {
@@ -221,20 +234,21 @@ public class CsvBulkLoadTool extends Configured implements Tool {
         
         List<TargetTableRef> tablesToBeLoaded = new ArrayList<TargetTableRef>();
         tablesToBeLoaded.add(new TargetTableRef(qualifiedTableName));
+        // using conn after it's been closed... o.O
         tablesToBeLoaded.addAll(getIndexTables(conn, schemaName, qualifiedTableName));
         
         // When loading a single index table, check index table name is correct
-        if(qualifedIndexTableName != null){
+        if (qualifiedIndexTableName != null){
             TargetTableRef targetIndexRef = null;
         	for (TargetTableRef tmpTable : tablesToBeLoaded){
-        		if(tmpTable.getLogicalName().compareToIgnoreCase(qualifedIndexTableName) == 0) {
+        		if (tmpTable.getLogicalName().compareToIgnoreCase(qualifiedIndexTableName) == 0) {
                     targetIndexRef = tmpTable;
         			break;
         		}
         	}
-        	if(targetIndexRef == null){
+        	if (targetIndexRef == null){
                 throw new IllegalStateException("CSV Bulk Loader error: index table " +
-                    qualifedIndexTableName + " doesn't exist");
+                    qualifiedIndexTableName + " doesn't exist");
         	}
         	tablesToBeLoaded.clear();
         	tablesToBeLoaded.add(targetIndexRef);
@@ -247,13 +261,14 @@ public class CsvBulkLoadTool extends Configured implements Tool {
                 .getProps()
                 .getBoolean(QueryServices.METRICS_ENABLED,
                         QueryServicesOptions.DEFAULT_IS_METRICS_ENABLED);
-        ExecutorService executor =  JobManager.createThreadPoolExec(Integer.MAX_VALUE, 5, 20, useInstrumentedPool);
+        ExecutorService executor =
+                JobManager.createThreadPoolExec(Integer.MAX_VALUE, 5, 20, useInstrumentedPool);
         try{
 	        for (TargetTableRef table : tablesToBeLoaded) {
 	        	Path tablePath = new Path(outputPath, table.getPhysicalName());
 	        	Configuration jobConf = new Configuration(conf);
 	        	jobConf.set(CsvToKeyValueMapper.TABLE_NAME_CONFKEY, qualifiedTableName);
-	        	if(qualifiedTableName.compareToIgnoreCase(table.getLogicalName()) != 0) {
+	        	if (qualifiedTableName.compareToIgnoreCase(table.getLogicalName()) != 0) {
                     jobConf.set(CsvToKeyValueMapper.INDEX_TABLE_NAME_CONFKEY, table.getPhysicalName());
 	        	}
 	        	TableLoader tableLoader = new TableLoader(
@@ -274,14 +289,6 @@ public class CsvBulkLoadTool extends Configured implements Tool {
 		return retCode;
 	}
 
-    String getJdbcUrl(String zkQuorum) {
-        if (zkQuorum == null) {
-            LOG.warn("Defaulting to localhost for ZooKeeper quorum");
-            zkQuorum = "localhost:2181";
-        }
-        return PhoenixRuntime.JDBC_PROTOCOL + PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR + zkQuorum;
-    }
-
     /**
      * Build up the list of columns to be imported. The list is taken from the command line if
      * present, otherwise it is taken from the table description.
@@ -327,9 +334,11 @@ public class CsvBulkLoadTool extends Configured implements Tool {
      * @param importColumns descriptors of columns to be imported
      * @param conf job configuration
      */
-    @VisibleForTesting
-    static void configureOptions(CommandLine cmdLine, List<ColumnInfo> importColumns,
-            Configuration conf) {
+    private static void configureOptions(CommandLine cmdLine, List<ColumnInfo> importColumns,
+            Configuration conf) throws SQLException {
+
+        // we don't parse ZK_QUORUM_OPT here because we need it in order to
+        // create the connection we need to build importColumns.
 
         char delimiterChar = ',';
         if (cmdLine.hasOption(DELIMITER_OPT.getOpt())) {
@@ -358,12 +367,6 @@ public class CsvBulkLoadTool extends Configured implements Tool {
             escapeChar = escapeString.charAt(0);
         }
 
-        if (cmdLine.hasOption(ZK_QUORUM_OPT.getOpt())) {
-            String zkQuorum = cmdLine.getOptionValue(ZK_QUORUM_OPT.getOpt());
-            LOG.info("Configuring ZK quorum to {}", zkQuorum);
-            conf.set(HConstants.ZOOKEEPER_QUORUM, zkQuorum);
-        }
-
         CsvBulkImportUtil.initCsvImportJob(
                 conf,
                 getQualifiedTableName(
@@ -493,7 +496,7 @@ public class CsvBulkLoadTool extends Configured implements Tool {
 	            job.setMapOutputKeyClass(ImmutableBytesWritable.class);
 	            job.setMapOutputValueClass(KeyValue.class);
 
-	            // initialize credentials to possibily run in a secure env
+	            // initialize credentials to possibly run in a secure env
 	            TableMapReduceUtil.initCredentials(job);
 
                 HTable htable = new HTable(conf, tableName);
@@ -522,8 +525,8 @@ public class CsvBulkLoadTool extends Configured implements Tool {
 	            }
 	            
 	            return true;
-            } catch(Exception ex) {
-            	LOG.error("Import job on table=" + tableName + " failed due to exception:" + ex);
+            } catch (Exception ex) {
+            	LOG.error("Import job on table=" + tableName + " failed due to exception.", ex);
             	return false;
             }
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c6b37b97/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/CsvToKeyValueMapper.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/CsvToKeyValueMapper.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/CsvToKeyValueMapper.java
index 90cb854..c0328bd 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/CsvToKeyValueMapper.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/CsvToKeyValueMapper.java
@@ -19,7 +19,6 @@ package org.apache.phoenix.mapreduce;
 
 import java.io.IOException;
 import java.io.StringReader;
-import java.sql.DriverManager;
 import java.sql.SQLException;
 import java.util.Iterator;
 import java.util.List;
@@ -32,7 +31,6 @@ import org.apache.commons.csv.CSVFormat;
 import org.apache.commons.csv.CSVParser;
 import org.apache.commons.csv.CSVRecord;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -41,11 +39,11 @@ import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.phoenix.jdbc.PhoenixConnection;
-import org.apache.phoenix.jdbc.PhoenixDriver;
 import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil;
 import org.apache.phoenix.util.CSVCommonsLoader;
 import org.apache.phoenix.util.ColumnInfo;
 import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.csv.CsvUpsertExecutor;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -108,7 +106,6 @@ public class CsvToKeyValueMapper extends Mapper<LongWritable,Text,ImmutableBytes
     protected void setup(Context context) throws IOException, InterruptedException {
 
         Configuration conf = context.getConfiguration();
-        String jdbcUrl = getJdbcUrl(conf);
 
         // pass client configuration into driver
         Properties clientInfos = new Properties();
@@ -118,12 +115,9 @@ public class CsvToKeyValueMapper extends Mapper<LongWritable,Text,ImmutableBytes
             clientInfos.setProperty(entry.getKey(), entry.getValue());
         }
         
-        // This statement also ensures that the driver class is loaded
-        LOG.info("Connection with driver {} with url {}", PhoenixDriver.class.getName(), jdbcUrl);
-
         try {
-            conn = (PhoenixConnection) DriverManager.getConnection(jdbcUrl, clientInfos);
-        } catch (SQLException e) {
+            conn = (PhoenixConnection) QueryUtil.getConnection(clientInfos, conf);
+        } catch (SQLException | ClassNotFoundException e) {
             throw new RuntimeException(e);
         }
 
@@ -189,20 +183,6 @@ public class CsvToKeyValueMapper extends Mapper<LongWritable,Text,ImmutableBytes
         }
     }
 
-    /**
-     * Build up the JDBC URL for connecting to Phoenix.
-     *
-     * @return the full JDBC URL for a Phoenix connection
-     */
-    @VisibleForTesting
-    static String getJdbcUrl(Configuration conf) {
-        String zkQuorum = conf.get(HConstants.ZOOKEEPER_QUORUM);
-        if (zkQuorum == null) {
-            throw new IllegalStateException(HConstants.ZOOKEEPER_QUORUM + " is not configured");
-        }
-        return PhoenixRuntime.JDBC_PROTOCOL + PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR + zkQuorum;
-    }
-
     @VisibleForTesting
     CsvUpsertExecutor buildUpsertExecutor(Configuration conf) {
         String tableName = conf.get(TABLE_NAME_CONFKEY);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c6b37b97/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index 30b43d5..b071bc5 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -807,7 +807,9 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         boolean isMetaTable = SchemaUtil.isMetaTable(tableName);
         boolean tableExist = true;
         try {
-            logger.info("Found quorum: " + ZKConfig.getZKQuorumServersString(config));
+            final String quorum = ZKConfig.getZKQuorumServersString(config);
+            final String znode = config.get(HConstants.ZOOKEEPER_ZNODE_PARENT);
+            logger.debug("Found quorum: " + quorum + ":" + znode);
             admin = new HBaseAdmin(config);
             try {
                 existingDesc = admin.getTableDescriptor(tableName);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c6b37b97/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java
index d63a68f..bd38983 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java
@@ -34,11 +34,13 @@ import javax.annotation.Nullable;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.util.Addressing;
 import org.apache.hadoop.hbase.zookeeper.ZKConfig;
 import org.apache.phoenix.iterate.ResultIterator;
 import org.apache.phoenix.jdbc.PhoenixDriver;
+import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver;
 import org.apache.phoenix.parse.HintNode;
 import org.apache.phoenix.parse.HintNode.Hint;
 import org.apache.phoenix.parse.WildcardParseNode;
@@ -49,8 +51,6 @@ import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
-import java.util.Iterator;
-import java.util.Map;
 
 public final class QueryUtil {
 
@@ -129,7 +129,7 @@ public final class QueryUtil {
      *
      * @param tableName name of the table for which the upsert statement is to be created
      * @param columns list of columns to be included in the upsert statement
-     * @param Hint hint to be added to the UPSERT statement.
+     * @param hint hint to be added to the UPSERT statement.
      * @return the created {@code UPSERT} statement
      */
     public static String constructUpsertStatement(String tableName, List<String> columns, Hint hint) {
@@ -222,13 +222,36 @@ public final class QueryUtil {
         return query.toString();
     }
 
-    public static String getUrl(String server) {
-        return PhoenixRuntime.JDBC_PROTOCOL + PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR + server;
+    /**
+     * Create the Phoenix JDBC connection URL from the provided cluster connection details.
+     */
+    public static String getUrl(String zkQuorum) {
+        return getUrlInternal(zkQuorum, null, null);
+    }
+
+    /**
+     * Create the Phoenix JDBC connection URL from the provided cluster connection details.
+     */
+    public static String getUrl(String zkQuorum, int clientPort) {
+        return getUrlInternal(zkQuorum, clientPort, null);
+    }
+
+    /**
+     * Create the Phoenix JDBC connection URL from the provided cluster connection details.
+     */
+    public static String getUrl(String zkQuorum, String znodeParent) {
+        return getUrlInternal(zkQuorum, null, znodeParent);
+    }
+
+    /**
+     * Create the Phoenix JDBC connection URL from the provided cluster connection details.
+     */
+    public static String getUrl(String zkQuorum, int port, String znodeParent) {
+        return getUrlInternal(zkQuorum, port, znodeParent);
     }
 
-    public static String getUrl(String server, long port) {
-        String serverUrl = getUrl(server);
-        return serverUrl + PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR + port
+    private static String getUrlInternal(String zkQuorum, Integer port, String znodeParent) {
+        return new PhoenixEmbeddedDriver.ConnectionInfo(zkQuorum, port, znodeParent).toUrl()
                 + PhoenixRuntime.JDBC_PROTOCOL_TERMINATOR;
     }
 
@@ -274,6 +297,7 @@ public final class QueryUtil {
 
     public static String getConnectionUrl(Properties props, Configuration conf)
             throws ClassNotFoundException, SQLException {
+        // TODO: props is ignored!
         // make sure we load the phoenix driver
         Class.forName(PhoenixDriver.class.getName());
 
@@ -304,12 +328,15 @@ public final class QueryUtil {
         if (port == -1) {
             port = conf.getInt(QueryServices.ZOOKEEPER_PORT_ATTRIB, -1);
             if (port == -1) {
+                // TODO: fall back to the default in HConstants#DEFAULT_ZOOKEPER_CLIENT_PORT
                 throw new RuntimeException("Client zk port was not set!");
             }
         }
         server = Joiner.on(',').join(servers);
+        String znodeParent = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT,
+                HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
 
-        return getUrl(server, port);
+        return getUrl(server, port, znodeParent);
     }
     
     public static String getViewStatement(String schemaName, String tableName, String where) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c6b37b97/phoenix-core/src/test/java/org/apache/phoenix/jdbc/PhoenixEmbeddedDriverTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/jdbc/PhoenixEmbeddedDriverTest.java b/phoenix-core/src/test/java/org/apache/phoenix/jdbc/PhoenixEmbeddedDriverTest.java
index 79f9ec6..083b205 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/jdbc/PhoenixEmbeddedDriverTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/jdbc/PhoenixEmbeddedDriverTest.java
@@ -40,9 +40,11 @@ public class PhoenixEmbeddedDriverTest {
             "jdbc:phoenix:localhost:123",
             "jdbc:phoenix:localhost:123;foo=bar",
             "jdbc:phoenix:localhost:123:/hbase",
-            "jdbc:phoenix:localhost:123:/hbase;foo=bas",
+            "jdbc:phoenix:localhost:123:/foo-bar",
+            "jdbc:phoenix:localhost:123:/foo-bar;foo=bas",
             "jdbc:phoenix:localhost:/hbase",
-            "jdbc:phoenix:localhost:/hbase;test=true",
+            "jdbc:phoenix:localhost:/foo-bar",
+            "jdbc:phoenix:localhost:/foo-bar;test=true",
             "jdbc:phoenix:v1,v2,v3",
             "jdbc:phoenix:v1,v2,v3;",
             "jdbc:phoenix:v1,v2,v3;test=true",
@@ -51,6 +53,7 @@ public class PhoenixEmbeddedDriverTest {
             "jdbc:phoenix:v1,v2,v3:123:/hbase",
             "jdbc:phoenix:v1,v2,v3:123:/hbase;test=false",
             "jdbc:phoenix:v1,v2,v3:123:/hbase:user/principal:/user.keytab;test=false",
+            "jdbc:phoenix:v1,v2,v3:123:/foo-bar:user/principal:/user.keytab;test=false",
             "jdbc:phoenix:v1,v2,v3:123:user/principal:/user.keytab;test=false",
             "jdbc:phoenix:v1,v2,v3:user/principal:/user.keytab;test=false",
             "jdbc:phoenix:v1,v2,v3:/hbase:user/principal:/user.keytab;test=false",
@@ -64,9 +67,11 @@ public class PhoenixEmbeddedDriverTest {
             new ConnectionInfo("localhost",123,null),
             new ConnectionInfo("localhost",123,null),
             new ConnectionInfo("localhost",123,"/hbase"),
-            new ConnectionInfo("localhost",123,"/hbase"),
-            new ConnectionInfo("localhost",null,"/hbase"),
+            new ConnectionInfo("localhost",123,"/foo-bar"),
+            new ConnectionInfo("localhost",123,"/foo-bar"),
             new ConnectionInfo("localhost",null,"/hbase"),
+            new ConnectionInfo("localhost",null,"/foo-bar"),
+            new ConnectionInfo("localhost",null,"/foo-bar"),
             new ConnectionInfo("v1,v2,v3",null,null),
             new ConnectionInfo("v1,v2,v3",null,null),
             new ConnectionInfo("v1,v2,v3",null,null),
@@ -75,6 +80,7 @@ public class PhoenixEmbeddedDriverTest {
             new ConnectionInfo("v1,v2,v3",123,"/hbase"),
             new ConnectionInfo("v1,v2,v3",123,"/hbase"),
             new ConnectionInfo("v1,v2,v3",123,"/hbase","user/principal", "/user.keytab" ),
+            new ConnectionInfo("v1,v2,v3",123,"/foo-bar","user/principal", "/user.keytab" ),
             new ConnectionInfo("v1,v2,v3",123, null,"user/principal", "/user.keytab" ),
             new ConnectionInfo("v1,v2,v3", null, null,"user/principal", "/user.keytab" ),
             new ConnectionInfo("v1,v2,v3",null,"/hbase","user/principal", "/user.keytab" ),

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c6b37b97/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/CsvBulkLoadToolTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/CsvBulkLoadToolTest.java b/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/CsvBulkLoadToolTest.java
index 31fc71c..33bb976 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/CsvBulkLoadToolTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/CsvBulkLoadToolTest.java
@@ -66,15 +66,4 @@ public class CsvBulkLoadToolTest {
     public void testGetQualifiedTableName_NullSchema() {
         assertEquals("MYTABLE", CsvBulkLoadTool.getQualifiedTableName(null, "myTable"));
     }
-
-    @Test
-    public void testGetJdbcUrl_WithQuorumSupplied() {
-        assertEquals("jdbc:phoenix:myzkhost:2181", bulkLoadTool.getJdbcUrl("myzkhost:2181"));
-    }
-
-    @Test
-    public void testGetJdbcUrl_NoQuorumSupplied() {
-        assertEquals("jdbc:phoenix:localhost:2181", bulkLoadTool.getJdbcUrl(null));
-    }
-
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c6b37b97/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/CsvToKeyValueMapperTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/CsvToKeyValueMapperTest.java b/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/CsvToKeyValueMapperTest.java
index 4033a65..dc6f497 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/CsvToKeyValueMapperTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/CsvToKeyValueMapperTest.java
@@ -95,21 +95,6 @@ public class CsvToKeyValueMapperTest {
     }
 
     @Test
-    public void testGetJdbcUrl() {
-        Configuration conf = new Configuration();
-        conf.set(HConstants.ZOOKEEPER_QUORUM, "myzkclient:2181");
-        String jdbcUrl = CsvToKeyValueMapper.getJdbcUrl(conf);
-
-        assertEquals("jdbc:phoenix:myzkclient:2181", jdbcUrl);
-    }
-
-    @Test(expected=IllegalStateException.class)
-    public void testGetJdbcUrl_NotConfigured() {
-        Configuration conf = new Configuration();
-        CsvToKeyValueMapper.getJdbcUrl(conf);
-    }
-
-    @Test
     public void testLoadPreUpdateProcessor() {
         Configuration conf = new Configuration();
         conf.setClass(PhoenixConfigurationUtil.UPSERT_HOOK_CLASS_CONFKEY, MockUpsertProcessor.class,

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c6b37b97/phoenix-core/src/test/java/org/apache/phoenix/util/QueryUtilTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/util/QueryUtilTest.java b/phoenix-core/src/test/java/org/apache/phoenix/util/QueryUtilTest.java
index beabaf1..8446e9e 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/util/QueryUtilTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/util/QueryUtilTest.java
@@ -17,10 +17,6 @@
  */
 package org.apache.phoenix.util;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
 import java.sql.Types;
 import java.util.Properties;
 
@@ -30,6 +26,8 @@ import org.junit.Test;
 
 import com.google.common.collect.ImmutableList;
 
+import static org.junit.Assert.*;
+
 public class QueryUtilTest {
 
     private static final ColumnInfo ID_COLUMN = new ColumnInfo("ID", Types.BIGINT);
@@ -96,19 +94,28 @@ public class QueryUtilTest {
     }
 
     private void validateUrl(String url) {
-        String prefix = QueryUtil.getUrl("");
+        String prefix = PhoenixRuntime.JDBC_PROTOCOL + PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR;
         assertTrue("JDBC URL missing jdbc protocol prefix", url.startsWith(prefix));
-        //remove the prefix, should only be left with server,server...:port
-        url = url.substring(prefix.length()+1);
-        // make sure only a single ':'
-        assertEquals("More than a single ':' in url: "+url, url.indexOf(PhoenixRuntime
-                .JDBC_PROTOCOL_SEPARATOR),
-                url.lastIndexOf(PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR));
+        assertTrue("JDBC URL missing jdbc terminator suffix", url.endsWith(";"));
+        // remove the prefix, should only be left with server[,server...]:port:/znode
+        url = url.substring(prefix.length());
+        String[] splits = url.split(":");
+        assertTrue("zk details should contain at least server component", splits.length >= 1);
         // make sure that each server is comma separated
-        url = url.substring(0, url.indexOf(PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR));
-        String[] servers = url.split(",");
+        String[] servers = splits[0].split(",");
         for(String server: servers){
             assertFalse("Found whitespace in server names for url: " + url, server.contains(" "));
         }
+        if (splits.length >= 2) {
+            // second bit is a port number, should not through
+            try {
+                Integer.parseInt(splits[1]);
+            } catch (NumberFormatException e) {
+                fail(e.getMessage());
+            }
+        }
+        if (splits.length >= 3) {
+            assertTrue("znode parent is not an absolute path", splits[2].startsWith("/"));
+        }
     }
 }
\ No newline at end of file


[15/31] phoenix git commit: PHOENIX-2007 java.sql.SQLException: Encountered exception in sub plan [0] execution(Alicia Ying Shu)

Posted by nd...@apache.org.
PHOENIX-2007 java.sql.SQLException: Encountered exception in sub plan [0] execution(Alicia Ying Shu)


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

Branch: refs/heads/4.x-HBase-1.1
Commit: eb9452d55068ff4574b48938aebba765c28caaaf
Parents: c1882ee
Author: Rajeshbabu Chintaguntla <ra...@apache.org>
Authored: Mon Jun 1 21:05:24 2015 +0530
Committer: Rajeshbabu Chintaguntla <ra...@apache.org>
Committed: Mon Jun 1 21:05:24 2015 +0530

----------------------------------------------------------------------
 .../org/apache/phoenix/end2end/HashJoinIT.java  | 54 ++++++++++++++++++++
 .../apache/phoenix/execute/HashJoinPlan.java    |  7 +--
 2 files changed, 58 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/eb9452d5/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinIT.java
index a03204a..88e03ca 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinIT.java
@@ -3813,6 +3813,60 @@ public class HashJoinIT extends BaseHBaseManagedTimeIT {
         }
     }
 
+    @Test
+    public void testSubqueryWithoutData() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        conn.setAutoCommit(false);
+
+        try {
+            String GRAMMAR_TABLE = "CREATE TABLE IF NOT EXISTS GRAMMAR_TABLE (ID INTEGER PRIMARY KEY, " +
+                    "unsig_id UNSIGNED_INT, big_id BIGINT, unsig_long_id UNSIGNED_LONG, tiny_id TINYINT," +
+                    "unsig_tiny_id UNSIGNED_TINYINT, small_id SMALLINT, unsig_small_id UNSIGNED_SMALLINT," + 
+                    "float_id FLOAT, unsig_float_id UNSIGNED_FLOAT, double_id DOUBLE, unsig_double_id UNSIGNED_DOUBLE," + 
+                    "decimal_id DECIMAL, boolean_id BOOLEAN, time_id TIME, date_id DATE, timestamp_id TIMESTAMP," + 
+                    "unsig_time_id TIME, unsig_date_id DATE, unsig_timestamp_id TIMESTAMP, varchar_id VARCHAR (30)," + 
+                    "char_id CHAR (30), binary_id BINARY (100), varbinary_id VARBINARY (100))";
+
+            String LARGE_TABLE = "CREATE TABLE IF NOT EXISTS LARGE_TABLE (ID INTEGER PRIMARY KEY, " +
+                    "unsig_id UNSIGNED_INT, big_id BIGINT, unsig_long_id UNSIGNED_LONG, tiny_id TINYINT," +
+                    "unsig_tiny_id UNSIGNED_TINYINT, small_id SMALLINT, unsig_small_id UNSIGNED_SMALLINT," + 
+                    "float_id FLOAT, unsig_float_id UNSIGNED_FLOAT, double_id DOUBLE, unsig_double_id UNSIGNED_DOUBLE," + 
+                    "decimal_id DECIMAL, boolean_id BOOLEAN, time_id TIME, date_id DATE, timestamp_id TIMESTAMP," + 
+                    "unsig_time_id TIME, unsig_date_id DATE, unsig_timestamp_id TIMESTAMP, varchar_id VARCHAR (30)," + 
+                    "char_id CHAR (30), binary_id BINARY (100), varbinary_id VARBINARY (100))";
+
+            String SECONDARY_LARGE_TABLE = "CREATE TABLE IF NOT EXISTS SECONDARY_LARGE_TABLE (SEC_ID INTEGER PRIMARY KEY," +
+                    "sec_unsig_id UNSIGNED_INT, sec_big_id BIGINT, sec_usnig_long_id UNSIGNED_LONG, sec_tiny_id TINYINT," + 
+                    "sec_unsig_tiny_id UNSIGNED_TINYINT, sec_small_id SMALLINT, sec_unsig_small_id UNSIGNED_SMALLINT," + 
+                    "sec_float_id FLOAT, sec_unsig_float_id UNSIGNED_FLOAT, sec_double_id DOUBLE, sec_unsig_double_id UNSIGNED_DOUBLE," +
+                    "sec_decimal_id DECIMAL, sec_boolean_id BOOLEAN, sec_time_id TIME, sec_date_id DATE," +
+                    "sec_timestamp_id TIMESTAMP, sec_unsig_time_id TIME, sec_unsig_date_id DATE, sec_unsig_timestamp_id TIMESTAMP," +
+                    "sec_varchar_id VARCHAR (30), sec_char_id CHAR (30), sec_binary_id BINARY (100), sec_varbinary_id VARBINARY (100))";
+            createTestTable(getUrl(), GRAMMAR_TABLE);
+            createTestTable(getUrl(), LARGE_TABLE);
+            createTestTable(getUrl(), SECONDARY_LARGE_TABLE);
+
+            String ddl = "SELECT * FROM (SELECT ID, BIG_ID, DATE_ID FROM LARGE_TABLE AS A WHERE (A.ID % 5) = 0) AS A " +
+                    "INNER JOIN (SELECT SEC_ID, SEC_TINY_ID, SEC_UNSIG_FLOAT_ID FROM SECONDARY_LARGE_TABLE AS B WHERE (B.SEC_ID % 5) = 0) AS B " +     
+                    "ON A.ID=B.SEC_ID WHERE A.DATE_ID > ALL (SELECT SEC_DATE_ID FROM SECONDARY_LARGE_TABLE LIMIT 100) " +      
+                    "AND B.SEC_UNSIG_FLOAT_ID = ANY (SELECT sec_unsig_float_id FROM SECONDARY_LARGE_TABLE " +                                       
+                    "WHERE SEC_ID > ALL (SELECT MIN (ID) FROM GRAMMAR_TABLE WHERE UNSIG_ID IS NULL) AND " +
+                    "SEC_UNSIG_ID < ANY (SELECT DISTINCT(UNSIG_ID) FROM LARGE_TABLE WHERE UNSIG_ID<2500) LIMIT 1000) " +
+                    "AND A.ID < 10000";
+            ResultSet rs = conn.createStatement().executeQuery(ddl);
+            assertFalse(rs.next());  
+        } finally {
+            Statement statement = conn.createStatement();
+            String query = "drop table GRAMMAR_TABLE";
+            statement.executeUpdate(query);
+            query = "drop table LARGE_TABLE";
+            statement.executeUpdate(query);
+            query = "drop table SECONDARY_LARGE_TABLE";
+            statement.executeUpdate(query);
+            conn.close();
+        }
+    }
 }
 
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/eb9452d5/phoenix-core/src/main/java/org/apache/phoenix/execute/HashJoinPlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/HashJoinPlan.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/HashJoinPlan.java
index aea075d..857a952 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/HashJoinPlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/HashJoinPlan.java
@@ -61,11 +61,12 @@ import org.apache.phoenix.parse.SelectStatement;
 import org.apache.phoenix.query.ConnectionQueryServices;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.schema.types.PArrayDataType;
 import org.apache.phoenix.schema.types.PBoolean;
 import org.apache.phoenix.schema.types.PDataType;
-import org.apache.phoenix.schema.PTable;
-import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.schema.types.PVarbinary;
 import org.apache.phoenix.util.SQLCloseable;
 import org.apache.phoenix.util.SQLCloseables;
 
@@ -254,7 +255,7 @@ public class HashJoinPlan extends DelegateQueryPlan {
             ImmutableBytesWritable ptr = new ImmutableBytesWritable();
             int columnCount = projector.getColumnCount();
             int rowCount = 0;
-            PDataType baseType = null;
+            PDataType baseType = PVarbinary.INSTANCE;
             for (Tuple tuple = iterator.next(); tuple != null; tuple = iterator.next()) {
                 if (expectSingleRow && rowCount >= 1)
                     throw new SQLExceptionInfo.Builder(SQLExceptionCode.SINGLE_ROW_SUBQUERY_RETURNS_MULTIPLE_ROWS).build().buildException();


[29/31] phoenix git commit: PHOENIX-1941 Phoenix tests are failing in linux env with missing class: StaticMapping (Alicia Ying Shu)

Posted by nd...@apache.org.
PHOENIX-1941 Phoenix tests are failing in linux env with missing class: StaticMapping (Alicia Ying Shu)


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

Branch: refs/heads/4.x-HBase-1.1
Commit: 329d74948521ed974593e455369a27d9cd705249
Parents: 52f5b04
Author: Nick Dimiduk <nd...@apache.org>
Authored: Wed Jun 17 12:17:33 2015 -0700
Committer: Nick Dimiduk <nd...@apache.org>
Committed: Wed Jun 17 12:23:47 2015 -0700

----------------------------------------------------------------------
 .../phoenix/end2end/End2EndTestDriver.java       | 19 +++++++++++++++----
 1 file changed, 15 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/329d7494/phoenix-core/src/it/java/org/apache/phoenix/end2end/End2EndTestDriver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/End2EndTestDriver.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/End2EndTestDriver.java
index 26d18cf..743f729 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/End2EndTestDriver.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/End2EndTestDriver.java
@@ -21,6 +21,7 @@ package org.apache.phoenix.end2end;
 
 import java.io.IOException;
 import java.io.PrintStream;
+import java.lang.annotation.Annotation;
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
@@ -79,10 +80,20 @@ public class End2EndTestDriver extends AbstractHBaseTool {
 
       @Override
       public boolean isCandidateClass(Class<?> c) {
-        return testFilterRe.matcher(c.getName()).find() &&
-          // Our pattern will match the below NON-IntegrationTest. Rather than
-          // do exotic regex, just filter it out here
-          super.isCandidateClass(c);
+          Annotation[] annotations = c.getAnnotations();
+          for (Annotation curAnnotation : annotations) {
+              if (curAnnotation.toString().contains("NeedsOwnMiniClusterTest")) {
+                  /* Skip tests that aren't designed to run against a live cluster.
+                   * For a live cluster, we cannot bring it up and down as required
+                   * for these tests to run.
+                   */
+                  return false;
+              }
+          }
+          return testFilterRe.matcher(c.getName()).find() &&
+                  // Our pattern will match the below NON-IntegrationTest. Rather than
+                  // do exotic regex, just filter it out here
+                  super.isCandidateClass(c);
       }
     }
 


[23/31] phoenix git commit: PHOENIX-2027 Subqueries with no data are raising IllegalStateException(Alicia Ying Shu)

Posted by nd...@apache.org.
PHOENIX-2027 Subqueries with no data are raising IllegalStateException(Alicia Ying Shu)


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

Branch: refs/heads/4.x-HBase-1.1
Commit: bfd860ffec62a784f1229997cf98892ea3c0592d
Parents: 18b9e72
Author: Rajeshbabu Chintaguntla <ra...@apache.org>
Authored: Wed Jun 10 01:01:29 2015 +0530
Committer: Rajeshbabu Chintaguntla <ra...@apache.org>
Committed: Wed Jun 10 01:01:29 2015 +0530

----------------------------------------------------------------------
 .../apache/phoenix/end2end/SortMergeJoinIT.java | 54 ++++++++++++++++++++
 .../phoenix/execute/SortMergeJoinPlan.java      |  4 +-
 2 files changed, 56 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/bfd860ff/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinIT.java
index 6f14a45..8b65ab3 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinIT.java
@@ -2658,5 +2658,59 @@ public class SortMergeJoinIT extends BaseHBaseManagedTimeIT {
         }
     }
 
+    @Test
+    public void testSubqueryWithoutData() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        conn.setAutoCommit(false);
+
+        try {
+            String GRAMMAR_TABLE = "CREATE TABLE IF NOT EXISTS GRAMMAR_TABLE (ID INTEGER PRIMARY KEY, " +
+                    "unsig_id UNSIGNED_INT, big_id BIGINT, unsig_long_id UNSIGNED_LONG, tiny_id TINYINT," +
+                    "unsig_tiny_id UNSIGNED_TINYINT, small_id SMALLINT, unsig_small_id UNSIGNED_SMALLINT," + 
+                    "float_id FLOAT, unsig_float_id UNSIGNED_FLOAT, double_id DOUBLE, unsig_double_id UNSIGNED_DOUBLE," + 
+                    "decimal_id DECIMAL, boolean_id BOOLEAN, time_id TIME, date_id DATE, timestamp_id TIMESTAMP," + 
+                    "unsig_time_id TIME, unsig_date_id DATE, unsig_timestamp_id TIMESTAMP, varchar_id VARCHAR (30)," + 
+                    "char_id CHAR (30), binary_id BINARY (100), varbinary_id VARBINARY (100))";
+
+            String LARGE_TABLE = "CREATE TABLE IF NOT EXISTS LARGE_TABLE (ID INTEGER PRIMARY KEY, " +
+                    "unsig_id UNSIGNED_INT, big_id BIGINT, unsig_long_id UNSIGNED_LONG, tiny_id TINYINT," +
+                    "unsig_tiny_id UNSIGNED_TINYINT, small_id SMALLINT, unsig_small_id UNSIGNED_SMALLINT," + 
+                    "float_id FLOAT, unsig_float_id UNSIGNED_FLOAT, double_id DOUBLE, unsig_double_id UNSIGNED_DOUBLE," + 
+                    "decimal_id DECIMAL, boolean_id BOOLEAN, time_id TIME, date_id DATE, timestamp_id TIMESTAMP," + 
+                    "unsig_time_id TIME, unsig_date_id DATE, unsig_timestamp_id TIMESTAMP, varchar_id VARCHAR (30)," + 
+                    "char_id CHAR (30), binary_id BINARY (100), varbinary_id VARBINARY (100))";
+
+            String SECONDARY_LARGE_TABLE = "CREATE TABLE IF NOT EXISTS SECONDARY_LARGE_TABLE (SEC_ID INTEGER PRIMARY KEY," +
+                    "sec_unsig_id UNSIGNED_INT, sec_big_id BIGINT, sec_usnig_long_id UNSIGNED_LONG, sec_tiny_id TINYINT," + 
+                    "sec_unsig_tiny_id UNSIGNED_TINYINT, sec_small_id SMALLINT, sec_unsig_small_id UNSIGNED_SMALLINT," + 
+                    "sec_float_id FLOAT, sec_unsig_float_id UNSIGNED_FLOAT, sec_double_id DOUBLE, sec_unsig_double_id UNSIGNED_DOUBLE," +
+                    "sec_decimal_id DECIMAL, sec_boolean_id BOOLEAN, sec_time_id TIME, sec_date_id DATE," +
+                    "sec_timestamp_id TIMESTAMP, sec_unsig_time_id TIME, sec_unsig_date_id DATE, sec_unsig_timestamp_id TIMESTAMP," +
+                    "sec_varchar_id VARCHAR (30), sec_char_id CHAR (30), sec_binary_id BINARY (100), sec_varbinary_id VARBINARY (100))";
+            createTestTable(getUrl(), GRAMMAR_TABLE);
+            createTestTable(getUrl(), LARGE_TABLE);
+            createTestTable(getUrl(), SECONDARY_LARGE_TABLE);
+
+            String ddl = "SELECT /*+USE_SORT_MERGE_JOIN*/ * FROM (SELECT ID, BIG_ID, DATE_ID FROM LARGE_TABLE AS A WHERE (A.ID % 5) = 0) AS A " +
+                    "INNER JOIN (SELECT SEC_ID, SEC_TINY_ID, SEC_UNSIG_FLOAT_ID FROM SECONDARY_LARGE_TABLE AS B WHERE (B.SEC_ID % 5) = 0) AS B " +     
+                    "ON A.ID=B.SEC_ID WHERE A.DATE_ID > ALL (SELECT SEC_DATE_ID FROM SECONDARY_LARGE_TABLE LIMIT 100) " +      
+                    "AND B.SEC_UNSIG_FLOAT_ID = ANY (SELECT sec_unsig_float_id FROM SECONDARY_LARGE_TABLE " +                                       
+                    "WHERE SEC_ID > ALL (SELECT MIN (ID) FROM GRAMMAR_TABLE WHERE UNSIG_ID IS NULL) AND " +
+                    "SEC_UNSIG_ID < ANY (SELECT DISTINCT(UNSIG_ID) FROM LARGE_TABLE WHERE UNSIG_ID<2500) LIMIT 1000) " +
+                    "AND A.ID < 10000";
+            ResultSet rs = conn.createStatement().executeQuery(ddl);
+            assertFalse(rs.next());  
+        } finally {
+            Statement statement = conn.createStatement();
+            String query = "drop table GRAMMAR_TABLE";
+            statement.executeUpdate(query);
+            query = "drop table LARGE_TABLE";
+            statement.executeUpdate(query);
+            query = "drop table SECONDARY_LARGE_TABLE";
+            statement.executeUpdate(query);
+            conn.close();
+        }
+    }
 }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bfd860ff/phoenix-core/src/main/java/org/apache/phoenix/execute/SortMergeJoinPlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/SortMergeJoinPlan.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/SortMergeJoinPlan.java
index 01e87e4..46ade33 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/SortMergeJoinPlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/SortMergeJoinPlan.java
@@ -53,11 +53,11 @@ import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.schema.KeyValueSchema;
+import org.apache.phoenix.schema.KeyValueSchema.KeyValueSchemaBuilder;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.TableRef;
 import org.apache.phoenix.schema.ValueBitSet;
-import org.apache.phoenix.schema.KeyValueSchema.KeyValueSchemaBuilder;
 import org.apache.phoenix.schema.tuple.ResultTuple;
 import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.util.ResultUtil;
@@ -485,7 +485,7 @@ public class SortMergeJoinPlan implements QueryPlan {
             this.expressions = expressions;
             this.keys = Lists.newArrayListWithExpectedSize(expressions.size());
             for (int i = 0; i < expressions.size(); i++) {
-                this.keys.add(new ImmutableBytesWritable());
+                this.keys.add(new ImmutableBytesWritable(EMPTY_PTR));
             }
         }
         


[03/31] phoenix git commit: PHOENIX-1763 Support building with HBase-1.1.0

Posted by nd...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/98271b88/phoenix-pig/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-pig/pom.xml b/phoenix-pig/pom.xml
index 2db1af6..015a660 100644
--- a/phoenix-pig/pom.xml
+++ b/phoenix-pig/pom.xml
@@ -54,7 +54,6 @@
     <dependency>
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-testing-util</artifactId>
-      <version>${hbase.version}</version>
       <scope>test</scope>
       <optional>true</optional>
       <exclusions>
@@ -67,7 +66,6 @@
     <dependency>
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-it</artifactId>
-      <version>${hbase.version}</version>
       <type>test-jar</type>
       <scope>test</scope>
       <exclusions>
@@ -80,41 +78,56 @@
     <dependency>
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-common</artifactId>
-      <version>${hbase.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-common</artifactId>
+      <scope>test</scope>
+      <type>test-jar</type>
     </dependency>
     <dependency>
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-protocol</artifactId>
-      <version>${hbase.version}</version>
     </dependency>
     <dependency>
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-client</artifactId>
-      <version>${hbase.version}</version>
+    </dependency>
+   <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-server</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-server</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-client</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-hadoop-compat</artifactId>
-      <version>${hbase.version}</version>
       <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-hadoop-compat</artifactId>
-      <version>${hbase.version}</version>
       <type>test-jar</type>
       <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-hadoop2-compat</artifactId>
-      <version>${hbase.version}</version>
       <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-hadoop2-compat</artifactId>
-      <version>${hbase.version}</version>
       <type>test-jar</type>
       <scope>test</scope>
     </dependency>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/98271b88/phoenix-spark/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-spark/pom.xml b/phoenix-spark/pom.xml
index adeed88..a232cf4 100644
--- a/phoenix-spark/pom.xml
+++ b/phoenix-spark/pom.xml
@@ -460,6 +460,13 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-server</artifactId>
+      <version>${hbase.version}</version>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-it</artifactId>
       <version>${hbase.version}</version>
       <type>test-jar</type>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/98271b88/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index d310c37..4361e54 100644
--- a/pom.xml
+++ b/pom.xml
@@ -78,7 +78,7 @@
     <test.output.tofile>true</test.output.tofile>
 
     <!-- Hadoop Versions -->
-    <hbase.version>1.0.1</hbase.version>
+    <hbase.version>1.1.0</hbase.version>
     <hadoop-two.version>2.5.1</hadoop-two.version>
 
     <!-- Dependency versions -->
@@ -452,6 +452,11 @@
       <!-- HBase dependencies -->
       <dependency>
         <groupId>org.apache.hbase</groupId>
+        <artifactId>hbase-annotations</artifactId>
+        <version>${hbase.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.hbase</groupId>
         <artifactId>hbase-testing-util</artifactId>
         <version>${hbase.version}</version>
         <scope>test</scope>
@@ -488,13 +493,34 @@
       </dependency>
       <dependency>
         <groupId>org.apache.hbase</groupId>
+        <artifactId>hbase-common</artifactId>
+        <version>${hbase.version}</version>
+        <type>test-jar</type>
+        <scope>test</scope>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.hbase</groupId>
         <artifactId>hbase-client</artifactId>
         <version>${hbase.version}</version>
       </dependency>
       <dependency>
         <groupId>org.apache.hbase</groupId>
+        <artifactId>hbase-client</artifactId>
+        <version>${hbase.version}</version>
+        <type>test-jar</type>
+        <scope>test</scope>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.hbase</groupId>
+        <artifactId>hbase-server</artifactId>
+        <version>${hbase.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.hbase</groupId>
         <artifactId>hbase-server</artifactId>
         <version>${hbase.version}</version>
+        <type>test-jar</type>
+        <scope>test</scope>
       </dependency>
       <dependency>
         <groupId>org.apache.hbase</groupId>
@@ -508,6 +534,19 @@
         <type>test-jar</type>
         <scope>test</scope>
       </dependency>
+      <dependency>
+        <groupId>org.apache.hbase</groupId>
+        <artifactId>hbase-hadoop2-compat</artifactId>
+        <version>${hbase.version}</version>
+        <scope>test</scope>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.hbase</groupId>
+        <artifactId>hbase-hadoop2-compat</artifactId>
+        <version>${hbase.version}</version>
+        <type>test-jar</type>
+        <scope>test</scope>
+      </dependency>
 
       <!-- Hadoop Dependencies -->
       <dependency>


[24/31] phoenix git commit: PHOENIX-1968: Should support saving arrays

Posted by nd...@apache.org.
PHOENIX-1968: Should support saving arrays


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

Branch: refs/heads/4.x-HBase-1.1
Commit: f7d734966f7172c3bc4a6f0ba31594ba74ee91a1
Parents: bfd860f
Author: ravimagham <ra...@apache.org>
Authored: Thu Jun 11 12:59:48 2015 -0700
Committer: ravimagham <ra...@apache.org>
Committed: Thu Jun 11 12:59:48 2015 -0700

----------------------------------------------------------------------
 .../apache/phoenix/spark/PhoenixSparkIT.scala   | 21 ++++++++++++++++
 .../phoenix/spark/PhoenixRecordWritable.scala   | 25 ++++++++++++++++----
 2 files changed, 41 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/f7d73496/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala
----------------------------------------------------------------------
diff --git a/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala b/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala
index 42e8676..5f256e6 100644
--- a/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala
+++ b/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala
@@ -415,4 +415,25 @@ class PhoenixSparkIT extends FunSuite with Matchers with BeforeAndAfterAll {
 
     results.toList shouldEqual checkResults
   }
+
+  test("Can save arrays back to phoenix") {
+    val dataSet = List((2L, Array("String1", "String2", "String3")))
+
+    sc
+      .parallelize(dataSet)
+      .saveToPhoenix(
+        "ARRAY_TEST_TABLE",
+        Seq("ID","VCARRAY"),
+        zkUrl = Some(quorumAddress)
+      )
+
+    // Load the results back
+    val stmt = conn.createStatement()
+    val rs = stmt.executeQuery("SELECT VCARRAY FROM ARRAY_TEST_TABLE WHERE ID = 2")
+    rs.next()
+    val sqlArray = rs.getArray(1).getArray().asInstanceOf[Array[String]]
+
+    // Verify the arrays are equal
+    sqlArray shouldEqual dataSet(0)._2
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f7d73496/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRecordWritable.scala
----------------------------------------------------------------------
diff --git a/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRecordWritable.scala b/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRecordWritable.scala
index 67e0bd2..3977657 100644
--- a/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRecordWritable.scala
+++ b/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRecordWritable.scala
@@ -16,11 +16,12 @@ package org.apache.phoenix.spark
 import java.sql.{PreparedStatement, ResultSet}
 import org.apache.hadoop.mapreduce.lib.db.DBWritable
 import org.apache.phoenix.mapreduce.util.ColumnInfoToStringEncoderDecoder
-import org.apache.phoenix.schema.types.{PDate, PhoenixArray}
+import org.apache.phoenix.schema.types.{PDataType, PDate, PhoenixArray}
 import org.joda.time.DateTime
 import scala.collection.{immutable, mutable}
 import scala.collection.JavaConversions._
 
+
 class PhoenixRecordWritable(var encodedColumns: String) extends DBWritable {
   val upsertValues = mutable.ArrayBuffer[Any]()
   val resultMap = mutable.Map[String, AnyRef]()
@@ -44,13 +45,27 @@ class PhoenixRecordWritable(var encodedColumns: String) extends DBWritable {
     upsertValues.zip(columns).zipWithIndex.foreach {
       case ((v, c), i) => {
         if (v != null) {
+
           // Both Java and Joda dates used to work in 4.2.3, but now they must be java.sql.Date
+          // Can override any other types here as needed
           val (finalObj, finalType) = v match {
-            case dt: DateTime => (new java.sql.Date(dt.getMillis), PDate.INSTANCE.getSqlType)
-            case d: java.util.Date => (new java.sql.Date(d.getTime), PDate.INSTANCE.getSqlType)
-            case _ => (v, c.getSqlType)
+            case dt: DateTime => (new java.sql.Date(dt.getMillis), PDate.INSTANCE)
+            case d: java.util.Date => (new java.sql.Date(d.getTime), PDate.INSTANCE)
+            case _ => (v, c.getPDataType)
+          }
+
+          // Save as array or object
+          finalObj match {
+            case obj: Array[AnyRef] => {
+              // Create a java.sql.Array, need to lookup the base sql type name
+              val sqlArray = statement.getConnection.createArrayOf(
+                PDataType.arrayBaseType(finalType).getSqlTypeName,
+                obj
+              )
+              statement.setArray(i + 1, sqlArray)
+            }
+            case _ => statement.setObject(i + 1, finalObj)
           }
-          statement.setObject(i + 1, finalObj, finalType)
         } else {
           statement.setNull(i + 1, c.getSqlType)
         }


[28/31] phoenix git commit: PHOENIX-2029 Queries are making two rpc calls for getTable

Posted by nd...@apache.org.
PHOENIX-2029 Queries are making two rpc calls for getTable


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

Branch: refs/heads/4.x-HBase-1.1
Commit: 52f5b04643914f33c2d00a1157ca767a32f1adb8
Parents: 43c722c
Author: Thomas D'Silva <tw...@gmail.com>
Authored: Mon Jun 8 15:30:40 2015 -0700
Committer: Thomas D'Silva <td...@salesforce.com>
Committed: Wed Jun 17 11:16:48 2015 -0700

----------------------------------------------------------------------
 .../org/apache/phoenix/rpc/UpdateCacheIT.java   | 139 +++++++++++++++++++
 .../apache/phoenix/compile/QueryCompiler.java   |   2 +-
 .../coprocessor/MetaDataEndpointImpl.java       |   6 +-
 .../apache/phoenix/schema/MetaDataClient.java   |  26 ++--
 4 files changed, 156 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/52f5b046/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheIT.java b/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheIT.java
new file mode 100644
index 0000000..c657e41
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheIT.java
@@ -0,0 +1,139 @@
+package org.apache.phoenix.rpc;
+
+import static org.apache.phoenix.util.TestUtil.INDEX_DATA_SCHEMA;
+import static org.apache.phoenix.util.TestUtil.MUTABLE_INDEX_DATA_TABLE;
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Matchers.isNull;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+import java.math.BigDecimal;
+import java.sql.Connection;
+import java.sql.Date;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Map;
+import java.util.Properties;
+
+import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.end2end.Shadower;
+import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver;
+import org.apache.phoenix.query.ConnectionQueryServices;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.MetaDataClient;
+import org.apache.phoenix.schema.PName;
+import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.util.DateUtil;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import com.google.common.collect.Maps;
+
+/**
+ * Verifies the number of rpcs calls from {@link MetaDataClient} updateCache() 
+ * for transactional and non-transactional tables.
+ */
+public class UpdateCacheIT extends BaseHBaseManagedTimeIT {
+	
+	public static final int NUM_MILLIS_IN_DAY = 86400000;
+
+    @Before
+    public void setUp() throws SQLException {
+        ensureTableCreated(getUrl(), MUTABLE_INDEX_DATA_TABLE);
+    }
+
+	@BeforeClass
+    @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
+    public static void doSetup() throws Exception {
+        Map<String,String> props = Maps.newHashMapWithExpectedSize(3);
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+    }
+	
+	public static void validateRowKeyColumns(ResultSet rs, int i) throws SQLException {
+		assertTrue(rs.next());
+		assertEquals(rs.getString(1), "varchar" + String.valueOf(i));
+		assertEquals(rs.getString(2), "char" + String.valueOf(i));
+		assertEquals(rs.getInt(3), i);
+		assertEquals(rs.getInt(4), i);
+		assertEquals(rs.getBigDecimal(5), new BigDecimal(i*0.5d));
+		Date date = new Date(DateUtil.parseDate("2015-01-01 00:00:00").getTime() + (i - 1) * NUM_MILLIS_IN_DAY);
+		assertEquals(rs.getDate(6), date);
+	}
+	
+	public static void setRowKeyColumns(PreparedStatement stmt, int i) throws SQLException {
+        // insert row
+        stmt.setString(1, "varchar" + String.valueOf(i));
+        stmt.setString(2, "char" + String.valueOf(i));
+        stmt.setInt(3, i);
+        stmt.setLong(4, i);
+        stmt.setBigDecimal(5, new BigDecimal(i*0.5d));
+        Date date = new Date(DateUtil.parseDate("2015-01-01 00:00:00").getTime() + (i - 1) * NUM_MILLIS_IN_DAY);
+        stmt.setDate(6, date);
+    }
+	
+	@Test
+	public void testUpdateCache() throws Exception {
+		String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + MUTABLE_INDEX_DATA_TABLE;
+		String selectSql = "SELECT * FROM "+fullTableName;
+		// use a spyed ConnectionQueryServices so we can verify calls to getTable
+		ConnectionQueryServices connectionQueryServices = Mockito.spy(driver.getConnectionQueryServices(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)));
+		Properties props = new Properties();
+		props.putAll(PhoenixEmbeddedDriver.DEFFAULT_PROPS.asMap());
+		Connection conn = connectionQueryServices.connect(getUrl(), props);
+		try {
+			conn.setAutoCommit(false);
+			ResultSet rs = conn.createStatement().executeQuery(selectSql);
+	     	assertFalse(rs.next());
+	     	reset(connectionQueryServices);
+	     	
+	        String upsert = "UPSERT INTO " + fullTableName + "(varchar_pk, char_pk, int_pk, long_pk, decimal_pk, date_pk) VALUES(?, ?, ?, ?, ?, ?)";
+	        PreparedStatement stmt = conn.prepareStatement(upsert);
+			// upsert three rows
+	        setRowKeyColumns(stmt, 1);
+			stmt.execute();
+			setRowKeyColumns(stmt, 2);
+			stmt.execute();
+			setRowKeyColumns(stmt, 3);
+			stmt.execute();
+			conn.commit();
+			// verify only one rpc to getTable occurs after commit is called
+			verify(connectionQueryServices, times(1)).getTable((PName)isNull(), eq(PVarchar.INSTANCE.toBytes(INDEX_DATA_SCHEMA)), eq(PVarchar.INSTANCE.toBytes(MUTABLE_INDEX_DATA_TABLE)), anyLong(), anyLong());
+			reset(connectionQueryServices);
+			
+			rs = conn.createStatement().executeQuery(selectSql);
+			validateRowKeyColumns(rs, 1);
+			validateRowKeyColumns(rs, 2);
+			validateRowKeyColumns(rs, 3);
+	        assertFalse(rs.next());
+	        
+	        rs = conn.createStatement().executeQuery(selectSql);
+	        validateRowKeyColumns(rs, 1);
+	        validateRowKeyColumns(rs, 2);
+	        validateRowKeyColumns(rs, 3);
+	        assertFalse(rs.next());
+	        
+	        rs = conn.createStatement().executeQuery(selectSql);
+	        validateRowKeyColumns(rs, 1);
+	        validateRowKeyColumns(rs, 2);
+	        validateRowKeyColumns(rs, 3);
+	        assertFalse(rs.next());
+	        conn.commit();
+	        // there should be one rpc to getTable per query
+	        verify(connectionQueryServices, times(3)).getTable((PName)isNull(), eq(PVarchar.INSTANCE.toBytes(INDEX_DATA_SCHEMA)), eq(PVarchar.INSTANCE.toBytes(MUTABLE_INDEX_DATA_TABLE)), anyLong(), anyLong());
+		}
+        finally {
+        	conn.close();
+        }
+	}
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/52f5b046/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java
index e877e03..94ff075 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java
@@ -529,7 +529,7 @@ public class QueryCompiler {
         // Don't pass groupBy when building where clause expression, because we do not want to wrap these
         // expressions as group by key expressions since they're pre, not post filtered.
         if (innerPlan == null && !tableRef.equals(resolver.getTables().get(0))) {
-            context.setResolver(FromCompiler.getResolverForQuery(select, this.statement.getConnection()));
+        	context.setResolver(FromCompiler.getResolver(context.getConnection(), tableRef, select.getUdfParseNodes()));
         }
         Set<SubqueryParseNode> subqueries = Sets.<SubqueryParseNode> newHashSet();
         Expression where = WhereCompiler.compile(context, select, viewWhere, subqueries);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/52f5b046/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
index 39a4956..1d578f5 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
@@ -1794,10 +1794,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                                     if (columnToDelete.isViewReferenced()) { // Disallow deletion of column referenced in WHERE clause of view
                                         return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION, EnvironmentEdgeManager.currentTimeMillis(), table, columnToDelete);
                                     }
-                                    // Look for columnToDelete in any indexes. If found as PK
-                                    // column, get lock and drop the index. If found as covered
-                                    // column, delete from index (do this client side?).
-                                    // In either case, invalidate index if the column is in it
+                                    // Look for columnToDelete in any indexes. If found as PK column, get lock and drop the index and then invalidate it
+                                    // Covered columns are deleted from the index by the client
                                     PhoenixConnection connection = table.getIndexes().isEmpty() ? null : QueryUtil.getConnection(env.getConfiguration()).unwrap(PhoenixConnection.class);
                                     for (PTable index : table.getIndexes()) {
                                         try {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/52f5b046/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
index fcdb651..75678fd 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
@@ -436,9 +436,9 @@ public class MetaDataClient {
                 // timestamp, we can handle this such that we don't ask the
                 // server again.
                 if (table != null) {
-                    // Ensures that table in result is set to table found in our cache.
-                    result.setTable(table);
                     if (code == MutationCode.TABLE_ALREADY_EXISTS) {
+                    	// Ensures that table in result is set to table found in our cache.
+                        result.setTable(table);
                         // Although this table is up-to-date, the parent table may not be.
                         // In this case, we update the parent table which may in turn pull
                         // in indexes to add to this table.
@@ -2692,18 +2692,20 @@ public class MetaDataClient {
 
                 dropColumnMutations(table, tableColumnsToDrop, tableMetaData);
                 for (PTable index : table.getIndexes()) {
+                	IndexMaintainer indexMaintainer = index.getIndexMaintainer(table, connection);
+                    // get the columns required for the index pk
+                    Set<ColumnReference> indexColumns = indexMaintainer.getIndexedColumns();
+                    // get the covered columns 
+                    Set<ColumnReference> coveredColumns = indexMaintainer.getCoverededColumns();
                     List<PColumn> indexColumnsToDrop = Lists.newArrayListWithExpectedSize(columnRefs.size());
                     for(PColumn columnToDrop : tableColumnsToDrop) {
-                        String indexColumnName = IndexUtil.getIndexColumnName(columnToDrop);
-                        try {
-                            PColumn indexColumn = index.getColumn(indexColumnName);
-                            if (SchemaUtil.isPKColumn(indexColumn)) {
-                                indexesToDrop.add(new TableRef(index));
-                            } else {
-                                indexColumnsToDrop.add(indexColumn);
-                                columnsToDrop.add(new ColumnRef(tableRef, columnToDrop.getPosition()));
-                            }
-                        } catch (ColumnNotFoundException e) {
+                        ColumnReference columnToDropRef = new ColumnReference(columnToDrop.getFamilyName().getBytes(), columnToDrop.getName().getBytes());
+						if (indexColumns.contains(columnToDropRef)) {
+                            indexesToDrop.add(new TableRef(index));
+                        } 
+                        else if (coveredColumns.contains(columnToDropRef)) {
+                        	String indexColumnName = IndexUtil.getIndexColumnName(columnToDrop);
+                            indexColumnsToDrop.add(index.getColumn(indexColumnName));
                         }
                     }
                     if(!indexColumnsToDrop.isEmpty()) {


[04/31] phoenix git commit: PHOENIX-1763 Support building with HBase-1.1.0

Posted by nd...@apache.org.
PHOENIX-1763 Support building with HBase-1.1.0


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

Branch: refs/heads/4.x-HBase-1.1
Commit: 98271b888c113f10e174205434e05d3b36b7eb67
Parents: bf01eb2
Author: Enis Soztutar <en...@apache.org>
Authored: Thu May 21 23:08:26 2015 -0700
Committer: Enis Soztutar <en...@apache.org>
Committed: Fri May 22 00:30:56 2015 -0700

----------------------------------------------------------------------
 phoenix-core/pom.xml                            | 17 +++--
 .../regionserver/IndexHalfStoreFileReader.java  | 31 ++++++--
 .../regionserver/IndexSplitTransaction.java     | 39 ++++++++--
 .../hbase/regionserver/LocalIndexMerger.java    |  3 +-
 .../cache/aggcache/SpillableGroupByCache.java   | 13 +++-
 .../phoenix/coprocessor/BaseRegionScanner.java  | 12 +--
 .../coprocessor/BaseScannerRegionObserver.java  | 77 +++++++++++---------
 .../coprocessor/DelegateRegionScanner.java      | 23 ++++--
 .../GroupedAggregateRegionObserver.java         | 53 ++++++++------
 .../coprocessor/HashJoinRegionScanner.java      | 60 ++++++++-------
 .../coprocessor/MetaDataRegionObserver.java     | 23 +++---
 .../phoenix/coprocessor/ScanRegionObserver.java | 11 ++-
 .../UngroupedAggregateRegionObserver.java       | 55 +++++++-------
 .../hbase/index/covered/data/LocalTable.java    |  2 +-
 .../index/covered/filter/FamilyOnlyFilter.java  |  6 +-
 .../index/scanner/FilteredKeyValueScanner.java  |  2 +-
 .../phoenix/index/PhoenixIndexBuilder.java      |  6 +-
 .../iterate/RegionScannerResultIterator.java    |  9 ++-
 .../phoenix/schema/stats/StatisticsScanner.java | 10 ++-
 .../hbase/ipc/PhoenixIndexRpcSchedulerTest.java |  6 +-
 .../index/covered/TestLocalTableState.java      |  1 -
 .../covered/filter/TestFamilyOnlyFilter.java    | 12 +--
 .../index/write/TestWALRecoveryCaching.java     |  4 +-
 phoenix-flume/pom.xml                           |  9 ---
 phoenix-pig/pom.xml                             | 31 +++++---
 phoenix-spark/pom.xml                           |  7 ++
 pom.xml                                         | 41 ++++++++++-
 27 files changed, 361 insertions(+), 202 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/98271b88/phoenix-core/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-core/pom.xml b/phoenix-core/pom.xml
index 45b8d73..22e6b60 100644
--- a/phoenix-core/pom.xml
+++ b/phoenix-core/pom.xml
@@ -350,16 +350,25 @@
     <dependency>
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-it</artifactId>
-      <version>${hbase.version}</version>
       <type>test-jar</type>
       <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-annotations</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-common</artifactId>
     </dependency>
     <dependency>
       <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-common</artifactId>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-protocol</artifactId>
     </dependency>
     <dependency>
@@ -369,18 +378,16 @@
     <dependency>
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-server</artifactId>
-      <version>${hbase.version}</version>
     </dependency>
     <dependency>
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-server</artifactId>
-      <version>${hbase.version}</version>
       <type>test-jar</type>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-hadoop-compat</artifactId>
-      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hbase</groupId>
@@ -391,13 +398,11 @@
     <dependency>
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-hadoop2-compat</artifactId>
-      <version>${hbase.version}</version>
       <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-hadoop2-compat</artifactId>
-      <version>${hbase.version}</version>
       <type>test-jar</type>
       <scope>test</scope>
     </dependency>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/98271b88/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReader.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReader.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReader.java
index 49e2022..9befc8c 100644
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReader.java
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReader.java
@@ -47,11 +47,11 @@ import org.apache.phoenix.index.IndexMaintainer;
  * that sort lowest and 'top' is the second half of the file with keys that sort greater than those
  * of the bottom half. The top includes the split files midkey, of the key that follows if it does
  * not exist in the file.
- * 
+ *
  * <p>
  * This type works in tandem with the {@link Reference} type. This class is used reading while
  * Reference is used writing.
- * 
+ *
  * <p>
  * This file is not splitable. Calls to {@link #midkey()} return null.
  */
@@ -64,7 +64,7 @@ public class IndexHalfStoreFileReader extends StoreFile.Reader {
     private final byte[] splitkey;
     private final byte[] splitRow;
     private final Map<ImmutableBytesWritable, IndexMaintainer> indexMaintainers;
-    private final byte[][] viewConstants; 
+    private final byte[][] viewConstants;
     private final int offset;
     private final HRegionInfo regionInfo;
     private final byte[] regionStartKeyInHFile;
@@ -144,6 +144,7 @@ public class IndexHalfStoreFileReader extends StoreFile.Reader {
             final HFileScanner delegate = s;
             public boolean atEnd = false;
 
+            @Override
             public ByteBuffer getKey() {
                 if (atEnd) {
                     return null;
@@ -160,7 +161,7 @@ public class IndexHalfStoreFileReader extends StoreFile.Reader {
                 // If it is top store file replace the StartKey of the Key with SplitKey
                 return getChangedKey(delegate.getKeyValue(), changeBottomKeys);
             }
-            
+
             private ByteBuffer getChangedKey(Cell kv, boolean changeBottomKeys) {
                 // new KeyValue(row, family, qualifier, timestamp, type, value)
                 byte[] newRowkey = getNewRowkeyByRegionStartKeyReplacedWithSplitKey(kv, changeBottomKeys);
@@ -183,6 +184,7 @@ public class IndexHalfStoreFileReader extends StoreFile.Reader {
                 return keyReplacedStartKey;
             }
 
+            @Override
             public String getKeyString() {
                 if (atEnd) {
                     return null;
@@ -190,6 +192,7 @@ public class IndexHalfStoreFileReader extends StoreFile.Reader {
                 return Bytes.toStringBinary(getKey());
             }
 
+            @Override
             public ByteBuffer getValue() {
                 if (atEnd) {
                     return null;
@@ -197,6 +200,7 @@ public class IndexHalfStoreFileReader extends StoreFile.Reader {
                 return delegate.getValue();
             }
 
+            @Override
             public String getValueString() {
                 if (atEnd) {
                     return null;
@@ -204,6 +208,7 @@ public class IndexHalfStoreFileReader extends StoreFile.Reader {
                 return Bytes.toStringBinary(getValue());
             }
 
+            @Override
             public Cell getKeyValue() {
                 if (atEnd) {
                     return null;
@@ -227,6 +232,7 @@ public class IndexHalfStoreFileReader extends StoreFile.Reader {
                 return changedKv;
             }
 
+            @Override
             public boolean next() throws IOException {
                 if (atEnd) {
                     return false;
@@ -248,10 +254,12 @@ public class IndexHalfStoreFileReader extends StoreFile.Reader {
                 }
             }
 
+            @Override
             public boolean seekBefore(byte[] key) throws IOException {
                 return seekBefore(key, 0, key.length);
             }
 
+            @Override
             public boolean seekBefore(byte[] key, int offset, int length) throws IOException {
 
                 if (top) {
@@ -282,6 +290,7 @@ public class IndexHalfStoreFileReader extends StoreFile.Reader {
                 return seekBefore(kv.getBuffer(), kv.getKeyOffset(), kv.getKeyLength());
             }
 
+            @Override
             public boolean seekTo() throws IOException {
                 boolean b = delegate.seekTo();
                 if (!b) {
@@ -302,10 +311,12 @@ public class IndexHalfStoreFileReader extends StoreFile.Reader {
                 }
             }
 
+            @Override
             public int seekTo(byte[] key) throws IOException {
                 return seekTo(key, 0, key.length);
             }
 
+            @Override
             public int seekTo(byte[] key, int offset, int length) throws IOException {
                 if (top) {
                     if (getComparator().compare(key, offset, length, splitkey, 0, splitkey.length) < 0) {
@@ -342,10 +353,12 @@ public class IndexHalfStoreFileReader extends StoreFile.Reader {
                 return seekTo(kv.getBuffer(), kv.getKeyOffset(), kv.getKeyLength());
             }
 
+            @Override
             public int reseekTo(byte[] key) throws IOException {
                 return reseekTo(key, 0, key.length);
             }
 
+            @Override
             public int reseekTo(byte[] key, int offset, int length) throws IOException {
                 if (top) {
                     if (getComparator().compare(key, offset, length, splitkey, 0, splitkey.length) < 0) {
@@ -375,11 +388,13 @@ public class IndexHalfStoreFileReader extends StoreFile.Reader {
                 return reseekTo(kv.getBuffer(), kv.getKeyOffset(), kv.getKeyLength());
             }
 
+            @Override
             public org.apache.hadoop.hbase.io.hfile.HFile.Reader getReader() {
                 return this.delegate.getReader();
             }
 
             // TODO: Need to change as per IndexHalfStoreFileReader
+            @Override
             public boolean isSeeked() {
                 return this.delegate.isSeeked();
             }
@@ -425,13 +440,13 @@ public class IndexHalfStoreFileReader extends StoreFile.Reader {
     /**
      * In case of top half store, the passed key will be with the start key of the daughter region.
      * But in the actual HFiles, the key will be with the start key of the old parent region. In
-     * order to make the real seek in the HFiles, we need to build the old key. 
-     * 
+     * order to make the real seek in the HFiles, we need to build the old key.
+     *
      * The logic here is just replace daughter region start key with parent region start key
      * in the key part.
-     * 
+     *
      * @param key
-     * 
+     *
      */
     private KeyValue getKeyPresentInHFiles(byte[] key) {
         KeyValue keyValue = new KeyValue(key);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/98271b88/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexSplitTransaction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexSplitTransaction.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexSplitTransaction.java
index 920380b..3057a14 100644
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexSplitTransaction.java
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexSplitTransaction.java
@@ -165,6 +165,7 @@ public class IndexSplitTransaction extends SplitTransaction {
    * @return <code>true</code> if the region is splittable else
    * <code>false</code> if it is not (e.g. its already closed, etc.).
    */
+  @Override
   public boolean prepare() {
     if (!this.parent.isSplittable()) return false;
     // Split key can be null if this region is unsplittable; i.e. has refs.
@@ -215,6 +216,7 @@ public class IndexSplitTransaction extends SplitTransaction {
    *    Call {@link #rollback(Server, RegionServerServices)}
    * @return Regions created
    */
+  @Override
   /* package */PairOfSameType<HRegion> createDaughters(final Server server,
       final RegionServerServices services) throws IOException {
     LOG.info("Starting split of region " + this.parent);
@@ -288,16 +290,19 @@ public class IndexSplitTransaction extends SplitTransaction {
       if (metaEntries == null || metaEntries.isEmpty()) {
         MetaTableAccessor.splitRegion(server.getConnection(), parent.getRegionInfo(),
                 daughterRegions.getFirst().getRegionInfo(),
-                daughterRegions.getSecond().getRegionInfo(), server.getServerName());
+                daughterRegions.getSecond().getRegionInfo(), server.getServerName(),
+                parent.getTableDesc().getRegionReplication());
       } else {
         offlineParentInMetaAndputMetaEntries(server.getConnection(),
           parent.getRegionInfo(), daughterRegions.getFirst().getRegionInfo(), daughterRegions
-              .getSecond().getRegionInfo(), server.getServerName(), metaEntries);
+              .getSecond().getRegionInfo(), server.getServerName(), metaEntries,
+              parent.getTableDesc().getRegionReplication());
       }
     }
     return daughterRegions;
   }
 
+  @Override
   public PairOfSameType<HRegion> stepsBeforePONR(final Server server,
       final RegionServerServices services, boolean testing) throws IOException {
     // Set ephemeral SPLITTING znode up in zk.  Mocked servers sometimes don't
@@ -380,6 +385,7 @@ public class IndexSplitTransaction extends SplitTransaction {
    * @throws IOException If thrown, transaction failed.
    *          Call {@link #rollback(Server, RegionServerServices)}
    */
+  @Override
   /* package */void openDaughters(final Server server,
       final RegionServerServices services, HRegion a, HRegion b)
       throws IOException {
@@ -565,6 +571,7 @@ public class IndexSplitTransaction extends SplitTransaction {
    * @throws IOException
    * @see #rollback(Server, RegionServerServices)
    */
+  @Override
   public PairOfSameType<HRegion> execute(final Server server,
       final RegionServerServices services)
   throws IOException {
@@ -575,6 +582,7 @@ public class IndexSplitTransaction extends SplitTransaction {
     return stepsAfterPONR(server, services, regions);
   }
 
+  @Override
   public PairOfSameType<HRegion> stepsAfterPONR(final Server server,
       final RegionServerServices services, PairOfSameType<HRegion> regions)
       throws IOException {
@@ -585,7 +593,7 @@ public class IndexSplitTransaction extends SplitTransaction {
 
   private void offlineParentInMetaAndputMetaEntries(Connection conn,
       HRegionInfo parent, HRegionInfo splitA, HRegionInfo splitB,
-      ServerName serverName, List<Mutation> metaEntries) throws IOException {
+      ServerName serverName, List<Mutation> metaEntries, int regionReplication) throws IOException {
     List<Mutation> mutations = metaEntries;
     HRegionInfo copyOfParent = new HRegionInfo(parent);
     copyOfParent.setOffline(true);
@@ -595,7 +603,7 @@ public class IndexSplitTransaction extends SplitTransaction {
     Put putParent = MetaTableAccessor.makePutFromRegionInfo(copyOfParent);
     MetaTableAccessor.addDaughtersToPut(putParent, splitA, splitB);
     mutations.add(putParent);
-    
+
     //Puts for daughters
     Put putA = MetaTableAccessor.makePutFromRegionInfo(splitA);
     Put putB = MetaTableAccessor.makePutFromRegionInfo(splitB);
@@ -604,9 +612,18 @@ public class IndexSplitTransaction extends SplitTransaction {
     addLocation(putB, serverName, 1);
     mutations.add(putA);
     mutations.add(putB);
+
+    // Add empty locations for region replicas of daughters so that number of replicas can be
+    // cached whenever the primary region is looked up from meta
+    for (int i = 1; i < regionReplication; i++) {
+      addEmptyLocation(putA, i);
+      addEmptyLocation(putB, i);
+    }
+
     MetaTableAccessor.mutateMetaTable(conn, mutations);
   }
 
+  @Override
   public Put addLocation(final Put p, final ServerName sn, long openSeqNum) {
     p.addImmutable(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
       Bytes.toBytes(sn.getHostAndPort()));
@@ -617,6 +634,13 @@ public class IndexSplitTransaction extends SplitTransaction {
     return p;
   }
 
+  private static Put addEmptyLocation(final Put p, int replicaId){
+    p.addImmutable(HConstants.CATALOG_FAMILY, MetaTableAccessor.getServerColumn(replicaId), null);
+    p.addImmutable(HConstants.CATALOG_FAMILY, MetaTableAccessor.getStartCodeColumn(replicaId), null);
+    p.addImmutable(HConstants.CATALOG_FAMILY, MetaTableAccessor.getSeqNumColumn(replicaId), null);
+    return p;
+  }
+
   /*
    * Open daughter region in its own thread.
    * If we fail, abort this hosting server.
@@ -659,6 +683,7 @@ public class IndexSplitTransaction extends SplitTransaction {
    * @throws IOException
    * @throws KeeperException
    */
+  @Override
   void openDaughterRegion(final Server server, final HRegion daughter)
   throws IOException, KeeperException {
     HRegionInfo hri = daughter.getRegionInfo();
@@ -767,6 +792,7 @@ public class IndexSplitTransaction extends SplitTransaction {
       this.family = family;
     }
 
+    @Override
     public Void call() throws IOException {
       splitStoreFile(family, sf);
       return null;
@@ -807,6 +833,7 @@ public class IndexSplitTransaction extends SplitTransaction {
    * @return True if we successfully rolled back, false if we got to the point
    * of no return and so now need to abort the server to minimize damage.
    */
+  @Override
   @SuppressWarnings("deprecation")
   public boolean rollback(final Server server, final RegionServerServices services)
   throws IOException {
@@ -879,10 +906,12 @@ public class IndexSplitTransaction extends SplitTransaction {
     return result;
   }
 
+  @Override
   HRegionInfo getFirstDaughter() {
     return hri_a;
   }
 
+  @Override
   HRegionInfo getSecondDaughter() {
     return hri_b;
   }
@@ -971,7 +1000,7 @@ public class IndexSplitTransaction extends SplitTransaction {
     return ZKAssign.transitionNode(zkw, parent, serverName,
       beginState, endState, znodeVersion, payload);
   }
-  
+
   public HRegion getParent() {
     return this.parent;
   }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/98271b88/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/LocalIndexMerger.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/LocalIndexMerger.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/LocalIndexMerger.java
index f074df7..add9b72 100644
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/LocalIndexMerger.java
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/LocalIndexMerger.java
@@ -81,7 +81,8 @@ public class LocalIndexMerger extends BaseRegionServerObserver {
                 this.mergedRegion = rmt.stepsBeforePONR(rss, rss, false);
                 rmt.prepareMutationsForMerge(mergedRegion.getRegionInfo(),
                     indexRegionA.getRegionInfo(), indexRegionB.getRegionInfo(),
-                    rss.getServerName(), metaEntries);
+                    rss.getServerName(), metaEntries,
+                    mergedRegion.getTableDesc().getRegionReplication());
             } catch (Exception e) {
                 ctx.bypass();
                 LOG.warn("index regions merge failed with the exception ", e);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/98271b88/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillableGroupByCache.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillableGroupByCache.java b/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillableGroupByCache.java
index ce18cc2..69fc6f6 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillableGroupByCache.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/cache/aggcache/SpillableGroupByCache.java
@@ -120,7 +120,7 @@ public class SpillableGroupByCache implements GroupByCache {
 
     /**
      * Instantiates a Loading LRU Cache that stores key / aggregator[] tuples used for group by queries
-     * 
+     *
      * @param estSize
      * @param estValueSize
      * @param aggs
@@ -325,7 +325,7 @@ public class SpillableGroupByCache implements GroupByCache {
 
     /**
      * Closes cache and releases spill resources
-     * 
+     *
      * @throws IOException
      */
     @Override
@@ -358,7 +358,9 @@ public class SpillableGroupByCache implements GroupByCache {
 
             @Override
             public boolean next(List<Cell> results) throws IOException {
-                if (!cacheIter.hasNext()) { return false; }
+                if (!cacheIter.hasNext()) {
+                    return false;
+                }
                 Map.Entry<ImmutableBytesWritable, Aggregator[]> ce = cacheIter.next();
                 ImmutableBytesWritable key = ce.getKey();
                 Aggregator[] aggs = ce.getValue();
@@ -377,6 +379,11 @@ public class SpillableGroupByCache implements GroupByCache {
             public long getMaxResultSize() {
               return s.getMaxResultSize();
             }
+
+            @Override
+            public int getBatch() {
+                return s.getBatch();
+            }
         };
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/98271b88/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseRegionScanner.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseRegionScanner.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseRegionScanner.java
index ff9ac76..828f776 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseRegionScanner.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseRegionScanner.java
@@ -22,14 +22,14 @@ import java.util.List;
 
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
-import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.regionserver.ScannerContext;
 
 public abstract class BaseRegionScanner implements RegionScanner {
 
     @Override
     public boolean isFilterDone() {
-        return false; 
+        return false;
     }
 
     @Override
@@ -38,10 +38,10 @@ public abstract class BaseRegionScanner implements RegionScanner {
     }
 
     @Override
-    public boolean next(List<Cell> result, int limit) throws IOException {
+    public boolean next(List<Cell> result, ScannerContext scannerContext) throws IOException {
         return next(result);
     }
-    
+
     @Override
     public boolean reseek(byte[] row) throws IOException {
         throw new DoNotRetryIOException("Unsupported");
@@ -58,7 +58,7 @@ public abstract class BaseRegionScanner implements RegionScanner {
     }
 
     @Override
-    public boolean nextRaw(List<Cell> result, int limit) throws IOException {
-        return next(result, limit);
+    public boolean nextRaw(List<Cell> result, ScannerContext scannerContext) throws IOException {
+        return next(result, scannerContext);
     }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/98271b88/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java
index a2269b4..fc74968 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java
@@ -37,6 +37,7 @@ 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.RegionScanner;
+import org.apache.hadoop.hbase.regionserver.ScannerContext;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.htrace.Span;
 import org.apache.htrace.Trace;
@@ -60,7 +61,7 @@ import com.google.common.collect.ImmutableList;
 
 
 abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
-    
+
     public static final String AGGREGATORS = "_Aggs";
     public static final String UNORDERED_GROUP_BY_EXPRESSIONS = "_UnorderedGroupByExpressions";
     public static final String KEY_ORDERED_GROUP_BY_EXPRESSIONS = "_OrderedGroupByExpressions";
@@ -91,7 +92,7 @@ abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
      * Attribute name used to pass custom annotations in Scans and Mutations (later). Custom annotations
      * are used to augment log lines emitted by Phoenix. See https://issues.apache.org/jira/browse/PHOENIX-1198.
      */
-    public static final String CUSTOM_ANNOTATIONS = "_Annot"; 
+    public static final String CUSTOM_ANNOTATIONS = "_Annot";
 
     /** Exposed for testing */
     public static final String SCANNER_OPENED_TRACE_INFO = "Scanner opened on server";
@@ -111,8 +112,8 @@ abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
     public String toString() {
         return this.getClass().getName();
     }
-    
-    
+
+
     private static void throwIfScanOutOfRegion(Scan scan, HRegion region) throws DoNotRetryIOException {
         boolean isLocalIndex = ScanUtil.isLocalIndex(scan);
         byte[] lowerInclusiveScanKey = scan.getStartRow();
@@ -136,7 +137,7 @@ abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
 
     abstract protected boolean isRegionObserverFor(Scan scan);
     abstract protected RegionScanner doPostScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c, final Scan scan, final RegionScanner s) throws Throwable;
-    
+
     @Override
     public RegionScanner preScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c,
         final Scan scan, final RegionScanner s) throws IOException {
@@ -153,7 +154,7 @@ abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
     /**
      * Wrapper for {@link #postScannerOpen(ObserverContext, Scan, RegionScanner)} that ensures no non IOException is thrown,
      * to prevent the coprocessor from becoming blacklisted.
-     * 
+     *
      */
     @Override
     public final RegionScanner postScannerOpen(
@@ -165,10 +166,10 @@ abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
             }
             boolean success =false;
             // Save the current span. When done with the child span, reset the span back to
-            // what it was. Otherwise, this causes the thread local storing the current span 
+            // what it was. Otherwise, this causes the thread local storing the current span
             // to not be reset back to null causing catastrophic infinite loops
             // and region servers to crash. See https://issues.apache.org/jira/browse/PHOENIX-1596
-            // TraceScope can't be used here because closing the scope will end up calling 
+            // TraceScope can't be used here because closing the scope will end up calling
             // currentSpan.stop() and that should happen only when we are closing the scanner.
             final Span savedSpan = Trace.currentSpan();
             final Span child = Trace.startSpan(SCANNER_OPENED_TRACE_INFO, savedSpan).getSpan();
@@ -226,7 +227,7 @@ abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
         return getWrappedScanner(c, s, null, null, offset, scan, dataColumns, tupleProjector,
                 dataRegion, indexMaintainer, viewConstants, null, null, projector, ptr);
     }
-    
+
     /**
      * Return wrapped scanner that catches unexpected exceptions (i.e. Phoenix bugs) and
      * re-throws as DoNotRetryIOException to prevent needless retrying hanging the query
@@ -246,7 +247,7 @@ abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
             final Expression[] arrayFuncRefs, final int offset, final Scan scan,
             final ColumnReference[] dataColumns, final TupleProjector tupleProjector,
             final HRegion dataRegion, final IndexMaintainer indexMaintainer,
-            final byte[][] viewConstants, final KeyValueSchema kvSchema, 
+            final byte[][] viewConstants, final KeyValueSchema kvSchema,
             final ValueBitSet kvSchemaBitSet, final TupleProjector projector,
             final ImmutableBytesWritable ptr) {
         return new RegionScanner() {
@@ -262,9 +263,9 @@ abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
             }
 
             @Override
-            public boolean next(List<Cell> result, int limit) throws IOException {
+            public boolean next(List<Cell> result, ScannerContext scannerContext) throws IOException {
                 try {
-                    return s.next(result, limit);
+                    return s.next(result, scannerContext);
                 } catch (Throwable t) {
                     ServerUtil.throwIOException(c.getEnvironment().getRegion().getRegionNameAsString(), t);
                     return false; // impossible
@@ -324,30 +325,31 @@ abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
             }
 
             @Override
-            public boolean nextRaw(List<Cell> result, int limit) throws IOException {
-                try {
-                    boolean next = s.nextRaw(result, limit);
-                    if (result.size() == 0) {
-                        return next;
-                    }
-                    if (arrayFuncRefs != null && arrayFuncRefs.length > 0 && arrayKVRefs.size() > 0) {
-                        replaceArrayIndexElement(arrayKVRefs, arrayFuncRefs, result);
-                    }
-                    if ((offset > 0 || ScanUtil.isLocalIndex(scan))  && !ScanUtil.isAnalyzeTable(scan)) {
-                        IndexUtil.wrapResultUsingOffset(c, result, offset, dataColumns,
-                            tupleProjector, dataRegion, indexMaintainer, viewConstants, ptr);
-                    }
-                    if (projector != null) {
-                        Tuple tuple = projector.projectResults(new ResultTuple(Result.create(result)));
-                        result.clear();
-                        result.add(tuple.getValue(0));
-                    }
-                    // There is a scanattribute set to retrieve the specific array element
+            public boolean nextRaw(List<Cell> result, ScannerContext scannerContext)
+                throws IOException {
+              try {
+                boolean next = s.nextRaw(result, scannerContext);
+                if (result.size() == 0) {
                     return next;
-                } catch (Throwable t) {
-                    ServerUtil.throwIOException(c.getEnvironment().getRegion().getRegionNameAsString(), t);
-                    return false; // impossible
                 }
+                if (arrayFuncRefs != null && arrayFuncRefs.length > 0 && arrayKVRefs.size() > 0) {
+                    replaceArrayIndexElement(arrayKVRefs, arrayFuncRefs, result);
+                }
+                if ((offset > 0 || ScanUtil.isLocalIndex(scan))  && !ScanUtil.isAnalyzeTable(scan)) {
+                    IndexUtil.wrapResultUsingOffset(c, result, offset, dataColumns,
+                        tupleProjector, dataRegion, indexMaintainer, viewConstants, ptr);
+                }
+                if (projector != null) {
+                    Tuple tuple = projector.projectResults(new ResultTuple(Result.create(result)));
+                    result.clear();
+                    result.add(tuple.getValue(0));
+                }
+                // There is a scanattribute set to retrieve the specific array element
+                return next;
+            } catch (Throwable t) {
+                ServerUtil.throwIOException(c.getEnvironment().getRegion().getRegionNameAsString(), t);
+                return false; // impossible
+            }
             }
 
             private void replaceArrayIndexElement(final Set<KeyValueColumnExpression> arrayKVRefs,
@@ -387,6 +389,11 @@ abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
             public long getMaxResultSize() {
                 return s.getMaxResultSize();
             }
+
+            @Override
+            public int getBatch() {
+                return s.getBatch();
+            }
         };
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/98271b88/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/DelegateRegionScanner.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/DelegateRegionScanner.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/DelegateRegionScanner.java
index f88a931..43c35a8 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/DelegateRegionScanner.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/DelegateRegionScanner.java
@@ -22,6 +22,7 @@ import java.util.List;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.regionserver.ScannerContext;
 
 public class DelegateRegionScanner implements RegionScanner {
 
@@ -56,23 +57,33 @@ public class DelegateRegionScanner implements RegionScanner {
         delegate.close();
     }
 
+    @Override
     public long getMaxResultSize() {
         return delegate.getMaxResultSize();
     }
 
-    public boolean next(List<Cell> arg0, int arg1) throws IOException {
-        return delegate.next(arg0, arg1);
+    @Override
+    public boolean next(List<Cell> result, ScannerContext scannerContext) throws IOException {
+        return delegate.next(result, scannerContext);
     }
 
-    public boolean next(List<Cell> arg0) throws IOException {
-        return delegate.next(arg0);
+    @Override
+    public boolean next(List<Cell> result) throws IOException {
+        return delegate.next(result);
     }
 
-    public boolean nextRaw(List<Cell> arg0, int arg1) throws IOException {
-        return delegate.nextRaw(arg0, arg1);
+    @Override
+    public boolean nextRaw(List<Cell> result, ScannerContext scannerContext) throws IOException {
+        return delegate.nextRaw(result, scannerContext);
     }
 
+    @Override
     public boolean nextRaw(List<Cell> arg0) throws IOException {
         return delegate.nextRaw(arg0);
     }
+
+    @Override
+    public int getBatch() {
+        return delegate.getBatch();
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/98271b88/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupedAggregateRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupedAggregateRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupedAggregateRegionObserver.java
index 1f1ba36..19a1663 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupedAggregateRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupedAggregateRegionObserver.java
@@ -80,7 +80,7 @@ import com.google.common.collect.Maps;
 
 /**
  * Region observer that aggregates grouped rows (i.e. SQL query with GROUP BY clause)
- * 
+ *
  * @since 0.1
  */
 public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
@@ -116,7 +116,7 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
             offset = region.getStartKey().length != 0 ? region.getStartKey().length:region.getEndKey().length;
             ScanUtil.setRowKeyOffset(scan, offset);
         }
-        
+
         List<Expression> expressions = deserializeGroupByExpressions(expressionBytes, 0);
         ServerAggregators aggregators =
                 ServerAggregators.deserialize(scan
@@ -124,7 +124,7 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
                         .getEnvironment().getConfiguration());
 
         RegionScanner innerScanner = s;
-        
+
         byte[] localIndexBytes = scan.getAttribute(LOCAL_INDEX_BUILD);
         List<IndexMaintainer> indexMaintainers = localIndexBytes == null ? null : IndexMaintainer.deserialize(localIndexBytes);
         TupleProjector tupleProjector = null;
@@ -142,9 +142,9 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
             }
             ImmutableBytesWritable tempPtr = new ImmutableBytesWritable();
             innerScanner =
-                    getWrappedScanner(c, innerScanner, offset, scan, dataColumns, tupleProjector, 
+                    getWrappedScanner(c, innerScanner, offset, scan, dataColumns, tupleProjector,
                             dataRegion, indexMaintainers == null ? null : indexMaintainers.get(0), viewConstants, p, tempPtr);
-        } 
+        }
 
         if (j != null) {
             innerScanner =
@@ -223,13 +223,13 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
     }
 
     /**
-     * 
+     *
      * Cache for distinct values and their aggregations which is completely
      * in-memory (as opposed to spilling to disk). Used when GROUPBY_SPILLABLE_ATTRIB
      * is set to false. The memory usage is tracked at a coursed grain and will
      * throw and abort if too much is used.
      *
-     * 
+     *
      * @since 3.0.0
      */
     private static final class InMemoryGroupByCache implements GroupByCache {
@@ -238,9 +238,9 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
         private final ServerAggregators aggregators;
         private final RegionCoprocessorEnvironment env;
         private final byte[] customAnnotations;
-        
+
         private int estDistVals;
-        
+
         InMemoryGroupByCache(RegionCoprocessorEnvironment env, ImmutableBytesWritable tenantId, byte[] customAnnotations, ServerAggregators aggregators, int estDistVals) {
             int estValueSize = aggregators.getEstimatedByteSize();
             long estSize = sizeOfUnorderedGroupByMap(estDistVals, estValueSize);
@@ -252,7 +252,7 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
             this.chunk = tenantCache.getMemoryManager().allocate(estSize);
             this.customAnnotations = customAnnotations;
         }
-        
+
         @Override
         public void close() throws IOException {
             this.chunk.close();
@@ -291,7 +291,7 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
             chunk.resize(estSize);
 
             final List<KeyValue> aggResults = new ArrayList<KeyValue>(aggregateMap.size());
-            
+
             final Iterator<Map.Entry<ImmutableBytesPtr, Aggregator[]>> cacheIter =
                     aggregateMap.entrySet().iterator();
             while (cacheIter.hasNext()) {
@@ -333,7 +333,9 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
 
                 @Override
                 public boolean next(List<Cell> results) throws IOException {
-                    if (index >= aggResults.size()) return false;
+                    if (index >= aggResults.size()) {
+                        return false;
+                    }
                     results.add(aggResults.get(index));
                     index++;
                     return index < aggResults.size();
@@ -343,6 +345,11 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
                 public long getMaxResultSize() {
                 	return s.getMaxResultSize();
                 }
+
+                @Override
+                public int getBatch() {
+                    return s.getBatch();
+                }
             };
         }
 
@@ -350,22 +357,22 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
         public long size() {
             return aggregateMap.size();
         }
-        
+
     }
     private static final class GroupByCacheFactory {
         public static final GroupByCacheFactory INSTANCE = new GroupByCacheFactory();
-        
+
         private GroupByCacheFactory() {
         }
-        
+
         GroupByCache newCache(RegionCoprocessorEnvironment env, ImmutableBytesWritable tenantId, byte[] customAnnotations, ServerAggregators aggregators, int estDistVals) {
             Configuration conf = env.getConfiguration();
             boolean spillableEnabled =
                     conf.getBoolean(GROUPBY_SPILLABLE_ATTRIB, DEFAULT_GROUPBY_SPILLABLE);
             if (spillableEnabled) {
                 return new SpillableGroupByCache(env, tenantId, aggregators, estDistVals);
-            } 
-            
+            }
+
             return new InMemoryGroupByCache(env, tenantId, customAnnotations, aggregators, estDistVals);
         }
     }
@@ -388,14 +395,14 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
         byte[] estDistValsBytes = scan.getAttribute(BaseScannerRegionObserver.ESTIMATED_DISTINCT_VALUES);
         if (estDistValsBytes != null) {
             // Allocate 1.5x estimation
-            estDistVals = Math.max(MIN_DISTINCT_VALUES, 
+            estDistVals = Math.max(MIN_DISTINCT_VALUES,
                             (int) (Bytes.toInt(estDistValsBytes) * 1.5f));
         }
 
         final boolean spillableEnabled =
                 conf.getBoolean(GROUPBY_SPILLABLE_ATTRIB, DEFAULT_GROUPBY_SPILLABLE);
 
-        GroupByCache groupByCache = 
+        GroupByCache groupByCache =
                 GroupByCacheFactory.INSTANCE.newCache(
                         env, ScanUtil.getTenantId(scan), ScanUtil.getCustomAnnotations(scan),
                         aggregators, estDistVals);
@@ -453,7 +460,7 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
      * Used for an aggregate query in which the key order match the group by key order. In this
      * case, we can do the aggregation as we scan, by detecting when the group by key changes.
      * @param limit TODO
-     * @throws IOException 
+     * @throws IOException
      */
     private RegionScanner scanOrdered(final ObserverContext<RegionCoprocessorEnvironment> c,
             final Scan scan, final RegionScanner scanner, final List<Expression> expressions,
@@ -559,11 +566,15 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
                 currentKey = null;
                 return false;
             }
-            
+
             @Override
             public long getMaxResultSize() {
                 return scanner.getMaxResultSize();
             }
+            @Override
+            public int getBatch() {
+                return scanner.getBatch();
+            }
         };
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/98271b88/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/HashJoinRegionScanner.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/HashJoinRegionScanner.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/HashJoinRegionScanner.java
index cdfc771..1e34d96 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/HashJoinRegionScanner.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/HashJoinRegionScanner.java
@@ -30,6 +30,7 @@ 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.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.regionserver.ScannerContext;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.cache.GlobalCache;
 import org.apache.phoenix.cache.HashCache;
@@ -48,7 +49,7 @@ import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.util.TupleUtil;
 
 public class HashJoinRegionScanner implements RegionScanner {
-    
+
     private final RegionScanner scanner;
     private final TupleProjector projector;
     private final HashJoinInfo joinInfo;
@@ -60,7 +61,7 @@ public class HashJoinRegionScanner implements RegionScanner {
     private List<Tuple>[] tempTuples;
     private ValueBitSet tempDestBitSet;
     private ValueBitSet[] tempSrcBitSet;
-    
+
     @SuppressWarnings("unchecked")
     public HashJoinRegionScanner(RegionScanner scanner, TupleProjector projector, HashJoinInfo joinInfo, ImmutableBytesWritable tenantId, RegionCoprocessorEnvironment env) throws IOException {
         this.scanner = scanner;
@@ -92,8 +93,8 @@ public class HashJoinRegionScanner implements RegionScanner {
             }
             HashCache hashCache = (HashCache)cache.getServerCache(joinId);
             if (hashCache == null)
-                throw new DoNotRetryIOException("Could not find hash cache for joinId: " 
-                        + Bytes.toString(joinId.get(), joinId.getOffset(), joinId.getLength()) 
+                throw new DoNotRetryIOException("Could not find hash cache for joinId: "
+                        + Bytes.toString(joinId.get(), joinId.getOffset(), joinId.getLength())
                         + ". The cache might have expired and have been removed.");
             hashCaches[i] = hashCache;
             tempSrcBitSet[i] = ValueBitSet.newInstance(joinInfo.getSchemas()[i]);
@@ -103,18 +104,19 @@ public class HashJoinRegionScanner implements RegionScanner {
             this.projector.setValueBitSet(tempDestBitSet);
         }
     }
-    
+
     private void processResults(List<Cell> result, boolean hasBatchLimit) throws IOException {
         if (result.isEmpty())
             return;
-        
+
         Tuple tuple = new ResultTuple(Result.create(result));
         // For backward compatibility. In new versions, HashJoinInfo.forceProjection()
         // always returns true.
         if (joinInfo.forceProjection()) {
             tuple = projector.projectResults(tuple);
         }
-        
+
+        // TODO: fix below Scanner.next() and Scanner.nextRaw() methods as well.
         if (hasBatchLimit)
             throw new UnsupportedOperationException("Cannot support join operations in scans with limit");
 
@@ -157,7 +159,7 @@ public class HashJoinRegionScanner implements RegionScanner {
                         Tuple lhs = resultQueue.poll();
                         if (!earlyEvaluation) {
                             ImmutableBytesPtr key = TupleUtil.getConcatenatedValue(lhs, joinInfo.getJoinExpressions()[i]);
-                            tempTuples[i] = hashCaches[i].get(key);                        	
+                            tempTuples[i] = hashCaches[i].get(key);
                             if (tempTuples[i] == null) {
                                 if (type == JoinType.Inner || type == JoinType.Semi) {
                                     continue;
@@ -171,7 +173,7 @@ public class HashJoinRegionScanner implements RegionScanner {
                             Tuple joined = tempSrcBitSet[i] == ValueBitSet.EMPTY_VALUE_BITSET ?
                                     lhs : TupleProjector.mergeProjectedValue(
                                             (ProjectedValueTuple) lhs, schema, tempDestBitSet,
-                                            null, joinInfo.getSchemas()[i], tempSrcBitSet[i], 
+                                            null, joinInfo.getSchemas()[i], tempSrcBitSet[i],
                                             joinInfo.getFieldPositions()[i]);
                             resultQueue.offer(joined);
                             continue;
@@ -180,7 +182,7 @@ public class HashJoinRegionScanner implements RegionScanner {
                             Tuple joined = tempSrcBitSet[i] == ValueBitSet.EMPTY_VALUE_BITSET ?
                                     lhs : TupleProjector.mergeProjectedValue(
                                             (ProjectedValueTuple) lhs, schema, tempDestBitSet,
-                                            t, joinInfo.getSchemas()[i], tempSrcBitSet[i], 
+                                            t, joinInfo.getSchemas()[i], tempSrcBitSet[i],
                                             joinInfo.getFieldPositions()[i]);
                             resultQueue.offer(joined);
                         }
@@ -211,18 +213,19 @@ public class HashJoinRegionScanner implements RegionScanner {
             }
         }
     }
-    
+
     private boolean shouldAdvance() {
         if (!resultQueue.isEmpty())
             return false;
-        
+
         return hasMore;
     }
-    
+
     private boolean nextInQueue(List<Cell> results) {
-        if (resultQueue.isEmpty())
+        if (resultQueue.isEmpty()) {
             return false;
-        
+        }
+
         Tuple tuple = resultQueue.poll();
         for (int i = 0; i < tuple.size(); i++) {
             results.add(tuple.getValue(i));
@@ -252,19 +255,19 @@ public class HashJoinRegionScanner implements RegionScanner {
             processResults(result, false);
             result.clear();
         }
-        
+
         return nextInQueue(result);
     }
 
     @Override
-    public boolean nextRaw(List<Cell> result, int limit)
+    public boolean nextRaw(List<Cell> result, ScannerContext scannerContext)
             throws IOException {
         while (shouldAdvance()) {
-            hasMore = scanner.nextRaw(result, limit);
-            processResults(result, true);
+            hasMore = scanner.nextRaw(result, scannerContext);
+            processResults(result, false); // TODO fix honoring the limit
             result.clear();
         }
-        
+
         return nextInQueue(result);
     }
 
@@ -285,19 +288,19 @@ public class HashJoinRegionScanner implements RegionScanner {
             processResults(result, false);
             result.clear();
         }
-        
+
         return nextInQueue(result);
     }
 
     @Override
-    public boolean next(List<Cell> result, int limit) throws IOException {
+    public boolean next(List<Cell> result, ScannerContext scannerContext) throws IOException {
         while (shouldAdvance()) {
-            hasMore = scanner.next(result, limit);
-            processResults(result, true);
+            hasMore = scanner.next(result, scannerContext);
+            processResults(result, false); // TODO honoring the limit
             result.clear();
         }
-        
-        return nextInQueue(result);
+
+      return nextInQueue(result);
     }
 
     @Override
@@ -305,5 +308,10 @@ public class HashJoinRegionScanner implements RegionScanner {
         return this.scanner.getMaxResultSize();
     }
 
+    @Override
+    public int getBatch() {
+        return this.scanner.getBatch();
+    }
+
 }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/98271b88/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
index 6f1d5ac..c40e3cd 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
@@ -69,20 +69,20 @@ public class MetaDataRegionObserver extends BaseRegionObserver {
     protected ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(1);
     private boolean enableRebuildIndex = QueryServicesOptions.DEFAULT_INDEX_FAILURE_HANDLING_REBUILD;
     private long rebuildIndexTimeInterval = QueryServicesOptions.DEFAULT_INDEX_FAILURE_HANDLING_REBUILD_INTERVAL;
-  
+
     @Override
     public void preClose(final ObserverContext<RegionCoprocessorEnvironment> c,
             boolean abortRequested) {
         executor.shutdownNow();
         GlobalCache.getInstance(c.getEnvironment()).getMetaDataCache().invalidateAll();
     }
-    
+
     @Override
     public void start(CoprocessorEnvironment env) throws IOException {
-        // sleep a little bit to compensate time clock skew when SYSTEM.CATALOG moves 
+        // sleep a little bit to compensate time clock skew when SYSTEM.CATALOG moves
         // among region servers because we relies on server time of RS which is hosting
         // SYSTEM.CATALOG
-        long sleepTime = env.getConfiguration().getLong(QueryServices.CLOCK_SKEW_INTERVAL_ATTRIB, 
+        long sleepTime = env.getConfiguration().getLong(QueryServices.CLOCK_SKEW_INTERVAL_ATTRIB,
             QueryServicesOptions.DEFAULT_CLOCK_SKEW_INTERVAL);
         try {
             if(sleepTime > 0) {
@@ -91,12 +91,12 @@ public class MetaDataRegionObserver extends BaseRegionObserver {
         } catch (InterruptedException ie) {
             Thread.currentThread().interrupt();
         }
-        enableRebuildIndex = env.getConfiguration().getBoolean(QueryServices.INDEX_FAILURE_HANDLING_REBUILD_ATTRIB, 
+        enableRebuildIndex = env.getConfiguration().getBoolean(QueryServices.INDEX_FAILURE_HANDLING_REBUILD_ATTRIB,
             QueryServicesOptions.DEFAULT_INDEX_FAILURE_HANDLING_REBUILD);
-        rebuildIndexTimeInterval = env.getConfiguration().getLong(QueryServices.INDEX_FAILURE_HANDLING_REBUILD_INTERVAL_ATTRIB, 
+        rebuildIndexTimeInterval = env.getConfiguration().getLong(QueryServices.INDEX_FAILURE_HANDLING_REBUILD_INTERVAL_ATTRIB,
             QueryServicesOptions.DEFAULT_INDEX_FAILURE_HANDLING_REBUILD_INTERVAL);
     }
-    
+
 
     @Override
     public void postOpen(ObserverContext<RegionCoprocessorEnvironment> e) {
@@ -119,7 +119,7 @@ public class MetaDataRegionObserver extends BaseRegionObserver {
             LOG.error("BuildIndexScheduleTask cannot start!", ex);
         }
     }
-    
+
     /**
      * Task runs periodically to build indexes whose INDEX_NEED_PARTIALLY_REBUILD is set true
      *
@@ -133,7 +133,7 @@ public class MetaDataRegionObserver extends BaseRegionObserver {
         public BuildIndexScheduleTask(RegionCoprocessorEnvironment env) {
             this.env = env;
         }
-      
+
         private String getJdbcUrl() {
             String zkQuorum = this.env.getConfiguration().get(HConstants.ZOOKEEPER_QUORUM);
             String zkClientPort = this.env.getConfiguration().get(HConstants.ZOOKEEPER_CLIENT_PORT,
@@ -144,7 +144,8 @@ public class MetaDataRegionObserver extends BaseRegionObserver {
                 + PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR + zkClientPort
                 + PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR + zkParentNode;
         }
-      
+
+        @Override
         public void run() {
             RegionScanner scanner = null;
             PhoenixConnection conn = null;
@@ -199,7 +200,7 @@ public class MetaDataRegionObserver extends BaseRegionObserver {
                         PhoenixDatabaseMetaData.INDEX_STATE_BYTES);
                     if ((dataTable == null || dataTable.length == 0)
                             || (indexStat == null || indexStat.length == 0)
-                            || ((Bytes.compareTo(PIndexState.DISABLE.getSerializedBytes(), indexStat) != 0) 
+                            || ((Bytes.compareTo(PIndexState.DISABLE.getSerializedBytes(), indexStat) != 0)
                                     && (Bytes.compareTo(PIndexState.INACTIVE.getSerializedBytes(), indexStat) != 0))) {
                         // index has to be either in disable or inactive state
                         // data table name can't be empty

http://git-wip-us.apache.org/repos/asf/phoenix/blob/98271b88/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ScanRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ScanRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ScanRegionObserver.java
index ddde407..77e124d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ScanRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ScanRegionObserver.java
@@ -199,7 +199,7 @@ public class ScanRegionObserver extends BaseScannerRegionObserver {
             indexMaintainer = indexMaintainers.get(0);
             viewConstants = IndexUtil.deserializeViewConstantsFromScan(scan);
         }
-        
+
         final TupleProjector p = TupleProjector.deserializeProjectorFromScan(scan);
         final HashJoinInfo j = HashJoinInfo.deserializeHashJoinFromScan(scan);
         innerScanner =
@@ -285,12 +285,12 @@ public class ScanRegionObserver extends BaseScannerRegionObserver {
                 } finally {
                     try {
                         if(iterator != null) {
-                            iterator.close();    
+                            iterator.close();
                         }
                     } catch (SQLException e) {
                         ServerUtil.throwIOException(region.getRegionNameAsString(), e);
                     } finally {
-                        chunk.close();                
+                        chunk.close();
                     }
                 }
             }
@@ -299,6 +299,11 @@ public class ScanRegionObserver extends BaseScannerRegionObserver {
             public long getMaxResultSize() {
                 return s.getMaxResultSize();
             }
+
+            @Override
+            public int getBatch() {
+              return s.getBatch();
+            }
         };
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/98271b88/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
index e43e5e5..2d6d98a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
@@ -101,8 +101,8 @@ import com.google.common.collect.Sets;
 
 /**
  * Region observer that aggregates ungrouped rows(i.e. SQL query with aggregation function and no GROUP BY).
- * 
- * 
+ *
+ *
  * @since 0.1
  */
 public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver{
@@ -116,7 +116,7 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver{
     public static final String EMPTY_CF = "EmptyCF";
     private static final Logger logger = LoggerFactory.getLogger(UngroupedAggregateRegionObserver.class);
     private KeyValueBuilder kvBuilder;
-    
+
     @Override
     public void start(CoprocessorEnvironment e) throws IOException {
         super.start(e);
@@ -139,14 +139,14 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver{
     public static void serializeIntoScan(Scan scan) {
         scan.setAttribute(BaseScannerRegionObserver.UNGROUPED_AGG, QueryConstants.TRUE);
     }
-    
+
     @Override
     public RegionScanner preScannerOpen(ObserverContext<RegionCoprocessorEnvironment> e, Scan scan, RegionScanner s)
             throws IOException {
         s = super.preScannerOpen(e, scan, s);
         if (ScanUtil.isAnalyzeTable(scan)) {
             // We are setting the start row and stop row such that it covers the entire region. As part
-            // of Phonenix-1263 we are storing the guideposts against the physical table rather than 
+            // of Phonenix-1263 we are storing the guideposts against the physical table rather than
             // individual tenant specific tables.
             scan.setStartRow(HConstants.EMPTY_START_ROW);
             scan.setStopRow(HConstants.EMPTY_END_ROW);
@@ -154,7 +154,7 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver{
         }
         return s;
     }
-    
+
     @Override
     protected RegionScanner doPostScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c, final Scan scan, final RegionScanner s) throws IOException {
         int offset = 0;
@@ -179,9 +179,9 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver{
         byte[] localIndexBytes = scan.getAttribute(LOCAL_INDEX_BUILD);
         List<IndexMaintainer> indexMaintainers = localIndexBytes == null ? null : IndexMaintainer.deserialize(localIndexBytes);
         List<Mutation> indexMutations = localIndexBytes == null ? Collections.<Mutation>emptyList() : Lists.<Mutation>newArrayListWithExpectedSize(1024);
-        
+
         RegionScanner theScanner = s;
-        
+
         byte[] indexUUID = scan.getAttribute(PhoenixIndexCodec.INDEX_UUID);
         PTable projectedTable = null;
         List<Expression> selectExpressions = null;
@@ -226,14 +226,14 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver{
             }
             ImmutableBytesWritable tempPtr = new ImmutableBytesWritable();
             theScanner =
-                    getWrappedScanner(c, theScanner, offset, scan, dataColumns, tupleProjector, 
+                    getWrappedScanner(c, theScanner, offset, scan, dataColumns, tupleProjector,
                             dataRegion, indexMaintainers == null ? null : indexMaintainers.get(0), viewConstants, p, tempPtr);
-        } 
-        
+        }
+
         if (j != null)  {
             theScanner = new HashJoinRegionScanner(theScanner, p, j, ScanUtil.getTenantId(scan), c.getEnvironment());
         }
-        
+
         int batchSize = 0;
         List<Mutation> mutations = Collections.emptyList();
         boolean buildLocalIndex = indexMaintainers != null && dataColumns==null && !localIndexScan;
@@ -330,7 +330,7 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver{
                                         }
                                         column.getDataType().coerceBytes(ptr, value,
                                             expression.getDataType(), expression.getMaxLength(),
-                                            expression.getScale(), expression.getSortOrder(), 
+                                            expression.getScale(), expression.getSortOrder(),
                                             column.getMaxLength(), column.getScale(),
                                             column.getSortOrder());
                                         byte[] bytes = ByteUtil.copyKeyBytesIfNecessary(ptr);
@@ -418,7 +418,7 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver{
                 }
             }
         }
-        
+
         if (logger.isDebugEnabled()) {
         	logger.debug(LogUtil.addCustomAnnotations("Finished scanning " + rowCount + " rows for ungrouped coprocessor scan " + scan, ScanUtil.getCustomAnnotations(scan)));
         }
@@ -438,7 +438,7 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver{
             keyValue = KeyValueUtil.newKeyValue(UNGROUPED_AGG_ROW_KEY, SINGLE_COLUMN_FAMILY, SINGLE_COLUMN, AGG_TIMESTAMP, value, 0, value.length);
         }
         final KeyValue aggKeyValue = keyValue;
-        
+
         RegionScanner scanner = new BaseRegionScanner() {
             private boolean done = !hadAny;
 
@@ -464,11 +464,16 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver{
                 results.add(aggKeyValue);
                 return false;
             }
-            
+
             @Override
             public long getMaxResultSize() {
             	return scan.getMaxResultSize();
             }
+
+            @Override
+            public int getBatch() {
+                return innerScanner.getBatch();
+            }
         };
         return scanner;
     }
@@ -496,7 +501,7 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver{
         }
         indexMutations.clear();
     }
-    
+
     @Override
     public InternalScanner preCompact(ObserverContext<RegionCoprocessorEnvironment> c,
         final Store store, InternalScanner scanner, final ScanType scanType)
@@ -505,8 +510,8 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver{
         InternalScanner internalScanner = scanner;
         if (scanType.equals(ScanType.COMPACT_DROP_DELETES)) {
             try {
-                boolean useCurrentTime = 
-                        c.getEnvironment().getConfiguration().getBoolean(QueryServices.STATS_USE_CURRENT_TIME_ATTRIB, 
+                boolean useCurrentTime =
+                        c.getEnvironment().getConfiguration().getBoolean(QueryServices.STATS_USE_CURRENT_TIME_ATTRIB,
                                 QueryServicesOptions.DEFAULT_STATS_USE_CURRENT_TIME);
                 // Provides a means of clients controlling their timestamps to not use current time
                 // when background tasks are updating stats. Instead we track the max timestamp of
@@ -526,8 +531,8 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver{
         }
         return internalScanner;
     }
-    
-    
+
+
     @Override
     public void postSplit(ObserverContext<RegionCoprocessorEnvironment> e, HRegion l, HRegion r)
             throws IOException {
@@ -535,8 +540,8 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver{
         TableName table = region.getRegionInfo().getTable();
         StatisticsCollector stats = null;
         try {
-            boolean useCurrentTime = 
-                    e.getEnvironment().getConfiguration().getBoolean(QueryServices.STATS_USE_CURRENT_TIME_ATTRIB, 
+            boolean useCurrentTime =
+                    e.getEnvironment().getConfiguration().getBoolean(QueryServices.STATS_USE_CURRENT_TIME_ATTRIB,
                             QueryServicesOptions.DEFAULT_STATS_USE_CURRENT_TIME);
             // Provides a means of clients controlling their timestamps to not use current time
             // when background tasks are updating stats. Instead we track the max timestamp of
@@ -544,7 +549,7 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver{
             long clientTimeStamp = useCurrentTime ? TimeKeeper.SYSTEM.getCurrentTime() : StatisticsCollector.NO_TIMESTAMP;
             stats = new StatisticsCollector(e.getEnvironment(), table.getNameAsString(), clientTimeStamp);
             stats.splitStats(region, l, r);
-        } catch (IOException ioe) { 
+        } catch (IOException ioe) {
             if(logger.isWarnEnabled()) {
                 logger.warn("Error while collecting stats during split for " + table,ioe);
             }
@@ -559,7 +564,7 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver{
             return PTableImpl.createFromProto(ptableProto);
         } catch (IOException e) {
             throw new RuntimeException(e);
-        } 
+        }
     }
 
     private static List<Expression> deserializeExpressions(byte[] b) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/98271b88/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/data/LocalTable.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/data/LocalTable.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/data/LocalTable.java
index 3469042..71cc1d6 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/data/LocalTable.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/data/LocalTable.java
@@ -70,4 +70,4 @@ public class LocalTable implements LocalHBaseState {
     scanner.close();
     return r;
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/98271b88/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/filter/FamilyOnlyFilter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/filter/FamilyOnlyFilter.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/filter/FamilyOnlyFilter.java
index 68555ef..d39b01d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/filter/FamilyOnlyFilter.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/filter/FamilyOnlyFilter.java
@@ -58,14 +58,14 @@ public class FamilyOnlyFilter extends FamilyFilter {
   @Override
   public ReturnCode filterKeyValue(Cell v) {
     if (done) {
-      return ReturnCode.SKIP;
+      return ReturnCode.NEXT_ROW;
     }
     ReturnCode code = super.filterKeyValue(v);
     if (previousMatchFound) {
       // we found a match before, and now we are skipping the key because of the family, therefore
       // we are done (no more of the family).
-      if (code.equals(ReturnCode.SKIP)) {
-      done = true;
+      if (code.equals(ReturnCode.SKIP) || code.equals(ReturnCode.NEXT_ROW)) {
+        done = true;
       }
     } else {
       // if we haven't seen a match before, then it doesn't matter what we see now, except to mark

http://git-wip-us.apache.org/repos/asf/phoenix/blob/98271b88/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/FilteredKeyValueScanner.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/FilteredKeyValueScanner.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/FilteredKeyValueScanner.java
index e225696..435a1c0 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/FilteredKeyValueScanner.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/FilteredKeyValueScanner.java
@@ -57,7 +57,7 @@ public class FilteredKeyValueScanner implements KeyValueScanner {
     /**
      * Same a {@link KeyValueScanner#next()} except that we filter out the next {@link KeyValue} until we find one that
      * passes the filter.
-     * 
+     *
      * @return the next {@link KeyValue} or <tt>null</tt> if no next {@link KeyValue} is present and passes all the
      *         filters.
      */

http://git-wip-us.apache.org/repos/asf/phoenix/blob/98271b88/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexBuilder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexBuilder.java b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexBuilder.java
index b89c807..b5e6a63 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexBuilder.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexBuilder.java
@@ -59,14 +59,14 @@ public class PhoenixIndexBuilder extends CoveredColumnsIndexBuilder {
             Mutation m = miniBatchOp.getOperation(i);
             keys.add(PVarbinary.INSTANCE.getKeyRange(m.getRow()));
             List<IndexMaintainer> indexMaintainers = getCodec().getIndexMaintainers(m.getAttributesMap());
-            
+
             for(IndexMaintainer indexMaintainer: indexMaintainers) {
                 if (indexMaintainer.isImmutableRows() && indexMaintainer.isLocalIndex()) continue;
                 indexTableName.set(indexMaintainer.getIndexTableName());
                 if (maintainers.get(indexTableName) != null) continue;
                 maintainers.put(indexTableName, indexMaintainer);
             }
-            
+
         }
         if (maintainers.isEmpty()) return;
         Scan scan = IndexManagementUtil.newLocalStateScan(new ArrayList<IndexMaintainer>(maintainers.values()));
@@ -100,7 +100,7 @@ public class PhoenixIndexBuilder extends CoveredColumnsIndexBuilder {
     private PhoenixIndexCodec getCodec() {
         return (PhoenixIndexCodec)this.codec;
     }
-    
+
     @Override
     public byte[] getBatchId(Mutation m){
         return this.codec.getBatchId(m);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/98271b88/phoenix-core/src/main/java/org/apache/phoenix/iterate/RegionScannerResultIterator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/RegionScannerResultIterator.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/RegionScannerResultIterator.java
index 88e141a..52fbe9c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/iterate/RegionScannerResultIterator.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/RegionScannerResultIterator.java
@@ -31,15 +31,15 @@ import org.apache.phoenix.util.ServerUtil;
 
 public class RegionScannerResultIterator extends BaseResultIterator {
     private final RegionScanner scanner;
-    
+
     public RegionScannerResultIterator(RegionScanner scanner) {
         this.scanner = scanner;
     }
-    
+
     @Override
     public Tuple next() throws SQLException {
-        // XXX: No access here to the region instance to enclose this with startRegionOperation / 
-        // stopRegionOperation 
+        // XXX: No access here to the region instance to enclose this with startRegionOperation /
+        // stopRegionOperation
         synchronized (scanner) {
             try {
                 // TODO: size
@@ -48,6 +48,7 @@ public class RegionScannerResultIterator extends BaseResultIterator {
                 // since this is an indication of whether or not there are more values after the
                 // ones returned
                 boolean hasMore = scanner.nextRaw(results);
+
                 if (!hasMore && results.isEmpty()) {
                     return null;
                 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/98271b88/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 de59304..0e50923 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
@@ -28,6 +28,7 @@ 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.phoenix.hbase.index.util.ImmutableBytesPtr;
 
 /**
@@ -58,15 +59,15 @@ public class StatisticsScanner implements InternalScanner {
     }
 
     @Override
-    public boolean next(List<Cell> result, int limit) throws IOException {
-        boolean ret = delegate.next(result, limit);
+    public boolean next(List<Cell> result, ScannerContext scannerContext) throws IOException {
+        boolean ret = delegate.next(result, scannerContext);
         updateStat(result);
         return ret;
     }
 
     /**
      * Update the current statistics based on the lastest batch of key-values from the underlying scanner
-     * 
+     *
      * @param results
      *            next batch of {@link KeyValue}s
      */
@@ -122,4 +123,5 @@ public class StatisticsScanner implements InternalScanner {
             }
         }
     }
-}
\ No newline at end of file
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/98271b88/phoenix-core/src/test/java/org/apache/hadoop/hbase/ipc/PhoenixIndexRpcSchedulerTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/hadoop/hbase/ipc/PhoenixIndexRpcSchedulerTest.java b/phoenix-core/src/test/java/org/apache/hadoop/hbase/ipc/PhoenixIndexRpcSchedulerTest.java
index 12f1863..030b114 100644
--- a/phoenix-core/src/test/java/org/apache/hadoop/hbase/ipc/PhoenixIndexRpcSchedulerTest.java
+++ b/phoenix-core/src/test/java/org/apache/hadoop/hbase/ipc/PhoenixIndexRpcSchedulerTest.java
@@ -27,6 +27,7 @@ import java.util.concurrent.TimeUnit;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.ipc.RpcScheduler.Context;
+import org.apache.hadoop.hbase.ipc.RpcServer.Connection;
 import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
 import org.junit.Test;
 import org.mockito.Mockito;
@@ -86,11 +87,12 @@ public class PhoenixIndexRpcSchedulerTest {
     }
 
     private void dispatchCallWithPriority(RpcScheduler scheduler, int priority) throws Exception {
+        Connection connection = Mockito.mock(Connection.class);
         CallRunner task = Mockito.mock(CallRunner.class);
         RequestHeader header = RequestHeader.newBuilder().setPriority(priority).build();
         RpcServer server = new RpcServer(null, "test-rpcserver", null, isa, conf, scheduler);
         RpcServer.Call call =
-                server.new Call(0, null, null, header, null, null, null, null, 10, null);
+                server.new Call(0, null, null, header, null, null, connection, null, 10, null, null);
         Mockito.when(task.getCall()).thenReturn(call);
 
         scheduler.dispatch(task);
@@ -98,4 +100,4 @@ public class PhoenixIndexRpcSchedulerTest {
         Mockito.verify(task).getCall();
         Mockito.verifyNoMoreInteractions(task);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/98271b88/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 54db5d8..e996b23 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
@@ -37,7 +37,6 @@ import org.junit.Test;
 import org.mockito.Mockito;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
-
 import org.apache.phoenix.hbase.index.covered.IndexUpdate;
 import org.apache.phoenix.hbase.index.covered.LocalTableState;
 import org.apache.phoenix.hbase.index.covered.data.LocalHBaseState;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/98271b88/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/filter/TestFamilyOnlyFilter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/filter/TestFamilyOnlyFilter.java b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/filter/TestFamilyOnlyFilter.java
index 216f548..808e6bc 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/filter/TestFamilyOnlyFilter.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/covered/filter/TestFamilyOnlyFilter.java
@@ -47,7 +47,7 @@ public class TestFamilyOnlyFilter {
 
     kv = new KeyValue(row, fam2, qual, 10, val);
     code = filter.filterKeyValue(kv);
-    assertEquals("Didn't filter out non-matching family!", ReturnCode.SKIP, code);
+    assertEquals("Didn't filter out non-matching family!", ReturnCode.NEXT_ROW, code);
   }
 
   @Test
@@ -61,7 +61,7 @@ public class TestFamilyOnlyFilter {
     KeyValue kv = new KeyValue(row, fam, qual, 10, val);
 
     ReturnCode code = filter.filterKeyValue(kv);
-    assertEquals("Didn't filter out non-matching family!", ReturnCode.SKIP, code);
+    assertEquals("Didn't filter out non-matching family!", ReturnCode.NEXT_ROW, code);
 
     kv = new KeyValue(row, fam2, qual, 10, val);
     code = filter.filterKeyValue(kv);
@@ -69,7 +69,7 @@ public class TestFamilyOnlyFilter {
 
     kv = new KeyValue(row, fam3, qual, 10, val);
     code = filter.filterKeyValue(kv);
-    assertEquals("Didn't filter out non-matching family!", ReturnCode.SKIP, code);
+    assertEquals("Didn't filter out non-matching family!", ReturnCode.NEXT_ROW, code);
   }
 
   @Test
@@ -83,7 +83,7 @@ public class TestFamilyOnlyFilter {
     KeyValue kv = new KeyValue(row, fam, qual, 10, val);
 
     ReturnCode code = filter.filterKeyValue(kv);
-    assertEquals("Didn't filter out non-matching family!", ReturnCode.SKIP, code);
+    assertEquals("Didn't filter out non-matching family!", ReturnCode.NEXT_ROW, code);
 
     KeyValue accept = new KeyValue(row, fam2, qual, 10, val);
     code = filter.filterKeyValue(accept);
@@ -91,12 +91,12 @@ public class TestFamilyOnlyFilter {
 
     kv = new KeyValue(row, fam3, qual, 10, val);
     code = filter.filterKeyValue(kv);
-    assertEquals("Didn't filter out non-matching family!", ReturnCode.SKIP, code);
+    assertEquals("Didn't filter out non-matching family!", ReturnCode.NEXT_ROW, code);
 
     // we shouldn't match the family again - everything after a switched family should be ignored
     code = filter.filterKeyValue(accept);
     assertEquals("Should have skipped a 'matching' family if it arrives out of order",
-      ReturnCode.SKIP, code);
+      ReturnCode.NEXT_ROW, code);
 
     // reset the filter and we should accept it again
     filter.reset();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/98271b88/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 60c11d7..ae577bd 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
@@ -317,9 +317,9 @@ public class TestWALRecoveryCaching {
         }
 
         LOG.info("Starting region server:" + server.getHostname());
-        cluster.startRegionServer(server.getHostname());
+        cluster.startRegionServer(server.getHostname(), server.getPort());
 
-        cluster.waitForRegionServerToStart(server.getHostname(), TIMEOUT);
+        cluster.waitForRegionServerToStart(server.getHostname(), server.getPort(), TIMEOUT);
 
         // start a server to get back to the base number of servers
         LOG.info("STarting server to replace " + server);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/98271b88/phoenix-flume/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-flume/pom.xml b/phoenix-flume/pom.xml
index 7ed0801..b2b9a47 100644
--- a/phoenix-flume/pom.xml
+++ b/phoenix-flume/pom.xml
@@ -85,7 +85,6 @@
     <dependency>
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-testing-util</artifactId>
-      <version>${hbase.version}</version>
       <scope>test</scope>
       <optional>true</optional>
       <exclusions>
@@ -98,7 +97,6 @@
     <dependency>
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-it</artifactId>
-      <version>${hbase.version}</version>
       <type>test-jar</type>
       <scope>test</scope>
       <exclusions>
@@ -111,41 +109,34 @@
     <dependency>
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-common</artifactId>
-      <version>${hbase.version}</version>
     </dependency>
     <dependency>
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-protocol</artifactId>
-      <version>${hbase.version}</version>
     </dependency>
     <dependency>
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-client</artifactId>
-      <version>${hbase.version}</version>
     </dependency>
     <dependency>
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-hadoop-compat</artifactId>
-      <version>${hbase.version}</version>
       <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-hadoop-compat</artifactId>
-      <version>${hbase.version}</version>
       <type>test-jar</type>
       <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-hadoop2-compat</artifactId>
-      <version>${hbase.version}</version>
       <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-hadoop2-compat</artifactId>
-      <version>${hbase.version}</version>
       <type>test-jar</type>
       <scope>test</scope>
     </dependency>


[30/31] phoenix git commit: PHOENIX-1935 org.apache.phoenix.end2end.ArithmeticQueryIT tests are failing (Alicia Ying Shu)

Posted by nd...@apache.org.
PHOENIX-1935 org.apache.phoenix.end2end.ArithmeticQueryIT tests are failing (Alicia Ying Shu)


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

Branch: refs/heads/4.x-HBase-1.1
Commit: 05b1b8b13f4137602567f67642946c883646d4d8
Parents: 329d749
Author: Nick Dimiduk <nd...@apache.org>
Authored: Wed Jun 17 12:28:35 2015 -0700
Committer: Nick Dimiduk <nd...@apache.org>
Committed: Wed Jun 17 12:31:28 2015 -0700

----------------------------------------------------------------------
 .../src/it/java/org/apache/phoenix/end2end/BaseViewIT.java      | 2 ++
 phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java | 3 +++
 .../src/test/java/org/apache/phoenix/query/BaseTest.java        | 5 ++++-
 3 files changed, 9 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/05b1b8b1/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewIT.java
index b9d7180..3140077 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewIT.java
@@ -98,6 +98,7 @@ public abstract class BaseViewIT extends BaseOwnClusterHBaseManagedTimeIT {
         assertEquals(1, rs.getInt(1));
         assertEquals(121, rs.getInt(2));
         assertFalse(rs.next());
+        conn.close();
     }
 
     protected void testUpdatableViewIndex(Integer saltBuckets) throws Exception {
@@ -179,6 +180,7 @@ public abstract class BaseViewIT extends BaseOwnClusterHBaseManagedTimeIT {
                                     + "CLIENT MERGE SORT",
                             QueryUtil.getExplainPlan(rs));
         }
+        conn.close();
     }
 
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/05b1b8b1/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
index 266438d..fb58a8f 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
@@ -92,8 +92,11 @@ public class ViewIT extends BaseViewIT {
             fail();
         } catch (ReadOnlyTableException e) {
             
+        } finally {
+            conn.close();
         }
 
+        conn = DriverManager.getConnection(getUrl());
         int count = 0;
         ResultSet rs = conn.createStatement().executeQuery("SELECT k FROM v2");
         while (rs.next()) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/05b1b8b1/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
index fa78656..3f09518 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
@@ -115,6 +115,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.IntegrationTestingUtility;
+import org.apache.hadoop.hbase.TableNotEnabledException;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.coprocessor.RegionServerObserver;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
@@ -1634,7 +1635,9 @@ public abstract class BaseTest {
             for (HTableDescriptor table : tables) {
                 String schemaName = SchemaUtil.getSchemaNameFromFullName(table.getName());
                 if (!QueryConstants.SYSTEM_SCHEMA_NAME.equals(schemaName)) {
-                    admin.disableTable(table.getName());
+                    try{
+                        admin.disableTable(table.getName());
+                    } catch (TableNotEnabledException ignored){}
                     admin.deleteTable(table.getName());
                 }
             }


[21/31] phoenix git commit: PHOENIX-777 - Support null value for fixed length ARRAY - Addendum (Ram)

Posted by nd...@apache.org.
PHOENIX-777 - Support null value for fixed length ARRAY - Addendum (Ram)


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

Branch: refs/heads/4.x-HBase-1.1
Commit: 58ee7062c624dd72a5cdaa41ec5b107a1e7b14c2
Parents: 6f890ad
Author: ramkrishna <ra...@gmail.com>
Authored: Tue Jun 2 14:32:02 2015 +0530
Committer: ramkrishna <ra...@gmail.com>
Committed: Tue Jun 2 14:36:05 2015 +0530

----------------------------------------------------------------------
 .../main/java/org/apache/phoenix/schema/types/PTimestamp.java   | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/58ee7062/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTimestamp.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTimestamp.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTimestamp.java
index d396adc..16b110e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTimestamp.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTimestamp.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.DateUtil;
 
 public class PTimestamp extends PDataType<Timestamp> {
@@ -47,6 +48,10 @@ public class PTimestamp extends PDataType<Timestamp> {
   @Override
   public int toBytes(Object object, byte[] bytes, int offset) {
     if (object == null) {
+      // Create the byte[] of size MAX_TIMESTAMP_BYTES
+      if(bytes.length != getByteSize()) {
+          bytes = Bytes.padTail(bytes, (getByteSize() - bytes.length));
+      }
       PDate.INSTANCE.getCodec().encodeLong(0l, bytes, offset);
       Bytes.putInt(bytes, offset + Bytes.SIZEOF_LONG, 0);
       return getByteSize();


[09/31] phoenix git commit: Revert "PHOENIX-2008 Integration tests are failing with HBase-1.1.0 because HBASE-13756(Rajeshbabu)"

Posted by nd...@apache.org.
Revert "PHOENIX-2008 Integration tests are failing with HBase-1.1.0 because HBASE-13756(Rajeshbabu)"

This reverts commit 56e1c0a1f348572fb73e9d0b8bbfb053df7f8710.


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

Branch: refs/heads/4.x-HBase-1.1
Commit: 1a2f2dc1ea6dd75c224b78a0dc2b312b0e6f5bce
Parents: 5546a42
Author: Rajeshbabu Chintaguntla <ra...@apache.org>
Authored: Wed May 27 14:52:25 2015 +0530
Committer: Rajeshbabu Chintaguntla <ra...@apache.org>
Committed: Wed May 27 14:52:25 2015 +0530

----------------------------------------------------------------------
 phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java | 2 --
 1 file changed, 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/1a2f2dc1/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
index 4aa28c4..54ae670 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
@@ -620,8 +620,6 @@ public abstract class BaseTest {
         }
         //no point doing sanity checks when running tests.
         conf.setBoolean("hbase.table.sanity.checks", false);
-        // Remove this configuration once hbase has HBASE-13756 fix.
-        conf.set("hbase.regionserver.msginterval", "300000");
         // set the server rpc controller and rpc scheduler factory, used to configure the cluster
         conf.set(RpcControllerFactory.CUSTOM_CONTROLLER_CONF_KEY, DEFAULT_SERVER_RPC_CONTROLLER_FACTORY);
         conf.set(RSRpcServices.REGION_SERVER_RPC_SCHEDULER_FACTORY_CLASS, DEFAULT_RPC_SCHEDULER_FACTORY);


[20/31] phoenix git commit: PHOENIX-777 - Support null value for fixed length ARRAY (Dumindu Buddhika)

Posted by nd...@apache.org.
PHOENIX-777 - Support null value for fixed length ARRAY (Dumindu Buddhika)


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

Branch: refs/heads/4.x-HBase-1.1
Commit: 6f890ade0691d03469ff8fce81c2fa9edd6941af
Parents: 9c5f111
Author: ramkrishna <ra...@gmail.com>
Authored: Tue Jun 2 11:18:51 2015 +0530
Committer: ramkrishna <ra...@gmail.com>
Committed: Tue Jun 2 11:18:51 2015 +0530

----------------------------------------------------------------------
 .../phoenix/end2end/ArraysWithNullsIT.java      | 300 +++++++++++++++++++
 .../phoenix/compile/ExpressionCompiler.java     |   9 +-
 .../apache/phoenix/schema/types/PBinary.java    |   2 +-
 .../org/apache/phoenix/schema/types/PChar.java  |   5 +-
 .../org/apache/phoenix/schema/types/PDate.java  |   6 +-
 .../apache/phoenix/schema/types/PDecimal.java   |   3 +
 .../apache/phoenix/schema/types/PTimestamp.java |  17 +-
 .../phoenix/schema/types/PhoenixArray.java      |  51 ++--
 8 files changed, 358 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/6f890ade/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArraysWithNullsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArraysWithNullsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArraysWithNullsIT.java
new file mode 100644
index 0000000..b034193
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArraysWithNullsIT.java
@@ -0,0 +1,300 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.phoenix.end2end;
+
+import static org.junit.Assert.assertEquals;
+
+import java.sql.*;
+
+import org.apache.phoenix.schema.types.PTimestamp;
+import org.apache.phoenix.schema.types.PhoenixArray;
+import org.junit.Test;
+
+public class ArraysWithNullsIT extends BaseClientManagedTimeIT {
+
+    @Test
+    public void testArrayUpsertIntWithNulls() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute("CREATE TABLE t1 ( k VARCHAR PRIMARY KEY, a INTEGER[])");
+
+        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO t1 VALUES('a',ARRAY[null,3,null])");
+        stmt.execute();
+        conn.commit();
+
+        ResultSet rs = conn.createStatement().executeQuery("Select a from t1 where k = 'a'");
+        rs.next();
+        Array array = conn.createArrayOf("INTEGER",new Object[]{null,3,null});
+
+        assertEquals(rs.getArray(1),array);
+        conn.close();
+
+    }
+
+
+
+    @Test
+    public void testArrayUpsertVarcharWithNulls() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute("CREATE TABLE t2 ( k VARCHAR PRIMARY KEY, a VARCHAR[])");
+
+        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO t2 VALUES('a',ARRAY['10',null])");
+        stmt.execute();
+        conn.commit();
+
+        ResultSet rs = conn.createStatement().executeQuery("Select a from t2 where k = 'a'");
+        rs.next();
+        Array array = conn.createArrayOf("VARCHAR",new Object[]{"10",null});
+
+        assertEquals(rs.getArray(1),array);
+        conn.close();
+
+    }
+
+    @Test
+    public void testArrayUpsertBigIntWithNulls() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute("CREATE TABLE t3 ( k VARCHAR PRIMARY KEY, a BIGINT[])");
+
+        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO t3 VALUES('a',ARRAY[2,null,32335,4])");
+        stmt.execute();
+        conn.commit();
+
+        ResultSet rs = conn.createStatement().executeQuery("Select a from t3 where k = 'a'");
+        rs.next();
+        Array array = conn.createArrayOf("BIGINT",new Object[]{(long)2,null,(long)32335,(long)4});
+
+        assertEquals(rs.getArray(1),array);
+        conn.close();
+
+    }
+
+    @Test
+    public void testArrayUpsertFloatWithNulls() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute("CREATE TABLE t4 ( k VARCHAR PRIMARY KEY, a FLOAT[])");
+
+        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO t4 VALUES('a',ARRAY[1.1,2.2,null,3.4])");
+        stmt.execute();
+        conn.commit();
+
+        ResultSet rs = conn.createStatement().executeQuery("Select a from t4 where k = 'a'");
+        rs.next();
+        Array array = conn.createArrayOf("FLOAT",new Object[]{(float)1.1,(float)2.2,null,(float)3.4});
+
+        assertEquals(rs.getArray(1),array);
+        conn.close();
+
+    }
+
+    @Test
+    public void testArrayUpsertSmallIntWithNulls() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute("CREATE TABLE t5 ( k VARCHAR PRIMARY KEY, a SMALLINT[])");
+
+        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO t5 VALUES('a',ARRAY[123,456,null,456])");
+        stmt.execute();
+        conn.commit();
+
+        ResultSet rs = conn.createStatement().executeQuery("Select a from t5 where k = 'a'");
+        rs.next();
+        Array array = conn.createArrayOf("SMALLINT",new Object[]{(short)123,(short)456,null,(short)456});
+
+        assertEquals(rs.getArray(1),array);
+        conn.close();
+
+    }
+
+    @Test
+    public void testArrayUpsertTinyIntWithNulls() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute("CREATE TABLE t6 ( k VARCHAR PRIMARY KEY, a TINYINT[])");
+
+        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO t6 VALUES('a',ARRAY[123,45,null,45])");
+        stmt.execute();
+        conn.commit();
+
+        ResultSet rs = conn.createStatement().executeQuery("Select a from t6 where k = 'a'");
+        rs.next();
+        Array array = conn.createArrayOf("TINYINT",new Object[]{(byte)123,(byte)45,null,(byte)45});
+
+        assertEquals(rs.getArray(1),array);
+        conn.close();
+
+    }
+
+    @Test
+    public void testArrayUpsertBooleanWithNulls() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute("CREATE TABLE t7 ( k VARCHAR PRIMARY KEY, a BOOLEAN[])");
+
+        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO t7 VALUES('a',ARRAY[true,false,null,true])");
+        stmt.execute();
+        conn.commit();
+
+        ResultSet rs = conn.createStatement().executeQuery("Select a from t7 where k = 'a'");
+        rs.next();
+        Array array = conn.createArrayOf("BOOLEAN",new Object[]{true,false,null,true});
+
+        assertEquals(rs.getArray(1),array);
+        conn.close();
+
+    }
+
+    @Test
+    public void testArrayUpsertDoubleWithNulls() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute("CREATE TABLE t8 ( k VARCHAR PRIMARY KEY, a DOUBLE[])");
+
+        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO t8 VALUES('a',ARRAY[1.2,2.3,null,3.4])");
+        stmt.execute();
+        conn.commit();
+
+        ResultSet rs = conn.createStatement().executeQuery("Select a from t8 where k = 'a'");
+        rs.next();
+        Array array = conn.createArrayOf("DOUBLE",new Object[]{1.2,2.3,null,3.4});
+
+        assertEquals(rs.getArray(1),array);
+        conn.close();
+
+    }
+
+    @Test
+    public void testArrayUpsertDateWithNulls1() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute("CREATE TABLE t9 ( k VARCHAR PRIMARY KEY, a DATE[])");
+        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO t9 VALUES('a',ARRAY[TO_DATE('2015-05-20 06:12:14.184'),null,TO_DATE('2015-05-20 06:12:14.184'),null])");
+        stmt.execute();
+        conn.commit();
+
+        ResultSet rs = conn.createStatement().executeQuery("Select a from t9 where k = 'a'");
+        rs.next();
+        Array array = conn.createArrayOf("DATE",new Date[]{new Date(1432102334184l),new Date(0l),new Date(1432102334184l),new Date(0l)});
+
+        assertEquals(rs.getArray(1),array);
+        conn.close();
+    }
+
+    @Test
+    public void testArrayUpsertDateWithNulls2() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute("CREATE TABLE t10 ( k VARCHAR PRIMARY KEY, a DATE[])");
+        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO t10 VALUES('a',ARRAY[TO_DATE('1970-01-01 00:00:00.000'), TO_DATE('2015-05-20 06:12:14.184'),TO_DATE('2015-05-20 06:12:14.184')])");
+        stmt.execute();
+        conn.commit();
+
+        ResultSet rs = conn.createStatement().executeQuery("Select a from t10 where k = 'a'");
+        rs.next();
+        Array array = conn.createArrayOf("DATE",new Date[]{new Date(0l), new Date(1432102334184l), new Date(1432102334184l)});
+
+        assertEquals(rs.getArray(1),array);
+        conn.close();
+    }
+
+    @Test
+    public void testArrayUpsertTimeWithNulls1() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute("CREATE TABLE t11 ( k VARCHAR PRIMARY KEY, a TIME[])");
+        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO t11 VALUES('a',ARRAY[TO_TIME('2015-05-20 06:12:14.184'),null,TO_TIME('2015-05-20 06:12:14.184'),null])");
+        stmt.execute();
+        conn.commit();
+
+        ResultSet rs = conn.createStatement().executeQuery("Select a from t11 where k = 'a'");
+        rs.next();
+        Array array = conn.createArrayOf("TIME",new Time[]{new Time(1432102334184l),new Time(0l),new Time(1432102334184l),new Time(0l)});
+
+        assertEquals(rs.getArray(1),array);
+        conn.close();
+    }
+
+    @Test
+    public void testArrayUpsertTimeWithNulls2() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute("CREATE TABLE t12 ( k VARCHAR PRIMARY KEY, a TIME[])");
+        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO t12 VALUES('a',ARRAY[TO_TIME('1970-01-01 00:00:00.000'), TO_TIME('2015-05-20 06:12:14.184'),null,TO_TIME('2015-05-20 06:12:14.184'),null])");
+        stmt.execute();
+        conn.commit();
+
+        ResultSet rs = conn.createStatement().executeQuery("Select a from t12 where k = 'a'");
+        rs.next();
+        Array array = conn.createArrayOf("TIME",new Time[]{new Time(0l),new Time(1432102334184l),new Time(0l),new Time(1432102334184l),new Time(0l)});
+
+        assertEquals(rs.getArray(1),array);
+        conn.close();
+    }
+
+    @Test
+    public void testArrayUpsertTimeStampWithNulls1() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute("CREATE TABLE t13 ( k VARCHAR PRIMARY KEY, a TIMESTAMP[])");
+        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO t13 VALUES('a',ARRAY[TO_TIMESTAMP('2015-05-20 06:12:14.184'),null,TO_TIMESTAMP('2015-05-20 06:12:14.184'),TO_TIMESTAMP('1970-01-01 00:00:00.000')])");
+        stmt.execute();
+        conn.commit();
+
+        ResultSet rs = conn.createStatement().executeQuery("Select a from t13 where k = 'a'");
+        rs.next();
+
+        assertEquals(rs.getArray(1),conn.createArrayOf("TIMESTAMP",new Timestamp[]{new Timestamp(1432102334184l),new Timestamp(0l),new Timestamp(1432102334184l),new Timestamp(0l)}));
+        conn.close();
+    }
+
+    @Test
+    public void testArrayUpsertTimeStampWithNulls2() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute("CREATE TABLE t14 ( k VARCHAR PRIMARY KEY, a TIMESTAMP[])");
+        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO t14 VALUES('a',ARRAY[TO_TIMESTAMP('1970-01-01 00:00:00.000'),TO_TIMESTAMP('2015-05-20 06:12:14.184'),TO_TIMESTAMP('1970-01-01 00:00:00.000'),TO_TIMESTAMP('2015-05-20 06:12:14.184'),TO_TIMESTAMP('1970-01-01 00:00:00.000')])");
+        stmt.execute();
+        conn.commit();
+
+        ResultSet rs = conn.createStatement().executeQuery("Select a from t14 where k = 'a'");
+        rs.next();
+
+        assertEquals(rs.getArray(1),conn.createArrayOf("TIMESTAMP",new Timestamp[]{new Timestamp(0l),new Timestamp(1432102334184l),new Timestamp(0l),new Timestamp(1432102334184l),new Timestamp(0l)}));
+        conn.close();
+    }
+
+    @Test
+    public void testArrayUpsertCharWithNulls1() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute("CREATE TABLE t15 ( k VARCHAR PRIMARY KEY, a CHAR(15)[])");
+        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO t15 VALUES('a',ARRAY['foo',null,'fo','foo'])");
+        stmt.execute();
+        conn.commit();
+
+        ResultSet rs = conn.createStatement().executeQuery("Select a from t15 where k = 'a'");
+        rs.next();
+
+        assertEquals(rs.getArray(1),conn.createArrayOf("CHAR",new String[]{"foo","","fo","foo"}));
+        conn.close();
+    }
+
+    @Test
+    public void testArrayUpsertCharWithNulls2() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute("CREATE TABLE t16 ( k VARCHAR PRIMARY KEY, a CHAR(15)[])");
+        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO t16 VALUES('a',ARRAY[null,'foo',null,'fo','foo'])");
+        stmt.execute();
+        conn.commit();
+
+        ResultSet rs = conn.createStatement().executeQuery("Select a from t16 where k = 'a'");
+        rs.next();
+
+        assertEquals(rs.getArray(1),conn.createArrayOf("CHAR",new String[]{"","foo","","fo","foo"}));
+        conn.close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6f890ade/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java
index 66c1b85..39baf7a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java
@@ -1271,8 +1271,13 @@ public class ExpressionCompiler extends UnsupportedAllParseNodeVisitor<Expressio
             for (int i = 0; i < children.size(); i++) {
                 Expression child = children.get(i);
                 child.evaluate(null, ptr);
-                Object value = arrayElemDataType.toObject(ptr, child.getDataType(), child.getSortOrder());
-                elements[i] = LiteralExpression.newConstant(value, child.getDataType(), child.getDeterminism()).getValue();
+                Object value = null;
+                if (child.getDataType() == null) {
+                    value = arrayElemDataType.toObject(ptr, theArrayElemDataType, child.getSortOrder());
+                } else {
+                    value = arrayElemDataType.toObject(ptr, child.getDataType(), child.getSortOrder());
+                }
+                elements[i] = LiteralExpression.newConstant(value, theArrayElemDataType, child.getDeterminism()).getValue();
             }
             Object value = PArrayDataType.instantiatePhoenixArray(arrayElemDataType, elements);
             return LiteralExpression.newConstant(value,

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6f890ade/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBinary.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBinary.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBinary.java
index d6d07fd..b397554 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBinary.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBinary.java
@@ -48,7 +48,7 @@ public class PBinary extends PDataType<byte[]> {
   public Object pad(Object object, Integer maxLength) {
     byte[] b = (byte[]) object;
     if (b == null) {
-      return null;
+      return new byte[maxLength];
     }
     if (b.length == maxLength) {
       return object;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6f890ade/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PChar.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PChar.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PChar.java
index 2effc38..c4d482c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PChar.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PChar.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.exception.DataExceedsCapacityException;
 import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.StringUtil;
 
 import com.google.common.base.Strings;
@@ -55,7 +56,7 @@ public class PChar extends PDataType<String> {
     public Object pad(Object object, Integer maxLength) {
       String s = (String) object;
       if (s == null) {
-        return s;
+        return Strings.padEnd("", maxLength, ' ');
       }
       if (s.length() == maxLength) {
         return object;
@@ -69,7 +70,7 @@ public class PChar extends PDataType<String> {
     @Override
     public byte[] toBytes(Object object) {
       if (object == null) {
-        throw newIllegalDataException(this + " may not be null");
+        return ByteUtil.EMPTY_BYTE_ARRAY;
       }
       byte[] b = PVarchar.INSTANCE.toBytes(object);
       if (b.length != ((String) object).length()) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6f890ade/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDate.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDate.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDate.java
index 947c30e..b10b1ac 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDate.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDate.java
@@ -39,9 +39,6 @@ public class PDate extends PDataType<Date> {
 
   @Override
   public byte[] toBytes(Object object) {
-    if (object == null) {
-      throw newIllegalDataException(this + " may not be null");
-    }
     byte[] bytes = new byte[getByteSize()];
     toBytes(object, bytes, 0);
     return bytes;
@@ -50,7 +47,8 @@ public class PDate extends PDataType<Date> {
   @Override
   public int toBytes(Object object, byte[] bytes, int offset) {
     if (object == null) {
-      throw newIllegalDataException(this + " may not be null");
+        getCodec().encodeLong(0l, bytes, offset);
+        return this.getByteSize();
     }
     getCodec().encodeLong(((java.util.Date) object).getTime(), bytes, offset);
     return this.getByteSize();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6f890ade/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDecimal.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDecimal.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDecimal.java
index 656113c..199ed28 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDecimal.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PDecimal.java
@@ -393,6 +393,9 @@ public class PDecimal extends PRealNumber<BigDecimal> {
   @Override
   public String toStringLiteral(Object o, Format formatter) {
       if (formatter == null) {
+          if(o == null) {
+              return String.valueOf(o);
+          }
           return ((BigDecimal)o).toPlainString();
         }
         return super.toStringLiteral(o, formatter);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6f890ade/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTimestamp.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTimestamp.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTimestamp.java
index 9a82cc0..d396adc 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTimestamp.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PTimestamp.java
@@ -39,9 +39,6 @@ public class PTimestamp extends PDataType<Timestamp> {
 
   @Override
   public byte[] toBytes(Object object) {
-    if (object == null) {
-      throw newIllegalDataException(this + " may not be null");
-    }
     byte[] bytes = new byte[getByteSize()];
     toBytes(object, bytes, 0);
     return bytes;
@@ -50,16 +47,18 @@ public class PTimestamp extends PDataType<Timestamp> {
   @Override
   public int toBytes(Object object, byte[] bytes, int offset) {
     if (object == null) {
-      throw newIllegalDataException(this + " may not be null");
+      PDate.INSTANCE.getCodec().encodeLong(0l, bytes, offset);
+      Bytes.putInt(bytes, offset + Bytes.SIZEOF_LONG, 0);
+      return getByteSize();
     }
     java.sql.Timestamp value = (java.sql.Timestamp) object;
     PDate.INSTANCE.getCodec().encodeLong(value.getTime(), bytes, offset);
 
-            /*
-             * By not getting the stuff that got spilled over from the millis part,
-             * it leaves the timestamp's byte representation saner - 8 bytes of millis | 4 bytes of nanos.
-             * Also, it enables timestamp bytes to be directly compared with date/time bytes.
-             */
+    /*
+     * By not getting the stuff that got spilled over from the millis part,
+     * it leaves the timestamp's byte representation saner - 8 bytes of millis | 4 bytes of nanos.
+     * Also, it enables timestamp bytes to be directly compared with date/time bytes.
+     */
     Bytes.putInt(bytes, offset + Bytes.SIZEOF_LONG, value.getNanos() % 1000000);
     return getByteSize();
   }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6f890ade/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PhoenixArray.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PhoenixArray.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PhoenixArray.java
index 9ffac83..843c831 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PhoenixArray.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PhoenixArray.java
@@ -23,6 +23,7 @@ import java.sql.SQLException;
 import java.util.Arrays;
 import java.util.Map;
 
+import com.google.common.base.Strings;
 import org.apache.phoenix.util.SQLCloseable;
 
 /**
@@ -54,11 +55,15 @@ public class PhoenixArray implements Array,SQLCloseable {
 	private static Object[] coerceToNewLength(PDataType baseType, Object[] elements, int maxLength) {
         Object[] resizedElements = new Object[elements.length];
         for (int i = 0; i < elements.length; i++) {
-            int length = baseType.getMaxLength(elements[i]);
-            if (length == maxLength) {
-                resizedElements[i] = elements[i];
+            Integer length = baseType.getMaxLength(elements[i]);
+            if (length != null) {
+                if (length == maxLength) {
+                    resizedElements[i] = elements[i];
+                } else {
+                    resizedElements[i] = baseType.pad(elements[i], maxLength);
+                }
             } else {
-                resizedElements[i] = baseType.pad(elements[i],maxLength);
+                resizedElements[i] = baseType.pad(elements[i], maxLength);
             }
         }
         return resizedElements;
@@ -67,18 +72,25 @@ public class PhoenixArray implements Array,SQLCloseable {
 	    if (elements == null || elements.length == 0) {
 	        return elements;
 	    }
-	    Object element = elements[0];
-	    int maxLength = baseType.getMaxLength(element);
-	    boolean resizeElements = false;
-	    for (int i = 1; i < elements.length; i++) {
-	        int length = baseType.getMaxLength(elements[i]);
-	        if (length > maxLength) {
-	            maxLength = length;
-	            resizeElements = true;
-	        } else if (length < maxLength) {
-	            resizeElements = true;
-	        }
-	    }
+        int maxLength = 0;
+        boolean resizeElements = false;
+        for (int i = 0; i < elements.length; i++) {
+            Integer length = baseType.getMaxLength(elements[i]);
+            if (length != null) {
+                if (maxLength == 0){
+                    maxLength = length;
+                    continue;
+                }
+                if (length > maxLength) {
+                    maxLength = length;
+                    resizeElements = true;
+                } else if (length < maxLength) {
+                    resizeElements = true;
+                }
+            } else {
+                resizeElements = true;
+            }
+        }
 	    if (!resizeElements) {
 	        return elements;
 	    }
@@ -92,7 +104,12 @@ public class PhoenixArray implements Array,SQLCloseable {
 		    if (baseType.getByteSize() == null) {
     		    elements = coerceToEqualLength(baseType, elements);
     		    if (elements != null && elements.length > 0) {
-    		        this.maxLength = baseType.getMaxLength(elements[0]);
+                    for(int i = 0; i < elements.length; i++) {
+                        if(elements[i] != null) {
+                            maxLength = baseType.getMaxLength(elements[i]);
+                            break;
+                        }
+                    }
     		    }
 		    } else {
 		        maxLength = baseType.getByteSize();


[18/31] phoenix git commit: PHOENIX-1962 Apply check style to the build

Posted by nd...@apache.org.
PHOENIX-1962 Apply check style to the build


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

Branch: refs/heads/4.x-HBase-1.1
Commit: 29ea503546265a619ce501c477a109b69f940a00
Parents: f2be913
Author: Nick Dimiduk <nd...@apache.org>
Authored: Sat May 9 11:10:54 2015 -0700
Committer: Nick Dimiduk <nd...@apache.org>
Committed: Mon Jun 1 12:21:48 2015 -0700

----------------------------------------------------------------------
 phoenix-assembly/pom.xml                    |   4 +
 phoenix-core/pom.xml                        |   4 +
 phoenix-flume/pom.xml                       |   4 +
 phoenix-pherf/pom.xml                       |   1 +
 phoenix-pig/pom.xml                         |   4 +
 phoenix-server-client/pom.xml               |   4 +
 phoenix-server/pom.xml                      |   4 +
 phoenix-spark/pom.xml                       |   1 +
 pom.xml                                     |  23 ++
 src/main/config/checkstyle/checker.xml      | 281 +++++++++++++++++++++++
 src/main/config/checkstyle/header.txt       |  16 ++
 src/main/config/checkstyle/suppressions.xml |  46 ++++
 12 files changed, 392 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/29ea5035/phoenix-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-assembly/pom.xml b/phoenix-assembly/pom.xml
index 04d9335..d275d03 100644
--- a/phoenix-assembly/pom.xml
+++ b/phoenix-assembly/pom.xml
@@ -33,6 +33,10 @@
   <description>Assemble Phoenix artifacts</description>
   <packaging>pom</packaging>
 
+  <properties>
+    <top.dir>${project.basedir}/..</top.dir>
+  </properties>
+
   <build>
     <plugins>
       <plugin>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/29ea5035/phoenix-core/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-core/pom.xml b/phoenix-core/pom.xml
index 951e969..6302441 100644
--- a/phoenix-core/pom.xml
+++ b/phoenix-core/pom.xml
@@ -24,6 +24,10 @@
       <url>http://www.apache.org</url>
   </organization>
 
+  <properties>
+    <top.dir>${project.basedir}/..</top.dir>
+  </properties>
+
   <build>
     <resources>
       <resource>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/29ea5035/phoenix-flume/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-flume/pom.xml b/phoenix-flume/pom.xml
index ea87ab0..c7f0650 100644
--- a/phoenix-flume/pom.xml
+++ b/phoenix-flume/pom.xml
@@ -31,6 +31,10 @@
   <artifactId>phoenix-flume</artifactId>
   <name>Phoenix - Flume</name>
 
+  <properties>
+    <top.dir>${project.basedir}/..</top.dir>
+  </properties>
+
   <dependencies>
    <dependency>
       <groupId>org.apache.phoenix</groupId>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/29ea5035/phoenix-pherf/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-pherf/pom.xml b/phoenix-pherf/pom.xml
index e751d73..dd45075 100644
--- a/phoenix-pherf/pom.xml
+++ b/phoenix-pherf/pom.xml
@@ -30,6 +30,7 @@
     <name>Phoenix - Pherf</name>
 
     <properties>
+      <top.dir>${project.basedir}/..</top.dir>
     </properties>
 
     <profiles>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/29ea5035/phoenix-pig/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-pig/pom.xml b/phoenix-pig/pom.xml
index 957c06f..55b34d3 100644
--- a/phoenix-pig/pom.xml
+++ b/phoenix-pig/pom.xml
@@ -31,6 +31,10 @@
   <artifactId>phoenix-pig</artifactId>
   <name>Phoenix - Pig</name>
 
+  <properties>
+    <top.dir>${project.basedir}/..</top.dir>
+  </properties>
+
   <dependencies>
     <dependency>
       <groupId>org.apache.phoenix</groupId>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/29ea5035/phoenix-server-client/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-server-client/pom.xml b/phoenix-server-client/pom.xml
index 748e57c..3e54a07 100644
--- a/phoenix-server-client/pom.xml
+++ b/phoenix-server-client/pom.xml
@@ -24,6 +24,10 @@
     <url>http://www.apache.org</url>
   </organization>
 
+  <properties>
+    <top.dir>${project.basedir}/..</top.dir>
+  </properties>
+
   <build>
     <plugins>
       <plugin>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/29ea5035/phoenix-server/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-server/pom.xml b/phoenix-server/pom.xml
index ab9a472..86b2525 100644
--- a/phoenix-server/pom.xml
+++ b/phoenix-server/pom.xml
@@ -24,6 +24,10 @@
     <url>http://www.apache.org</url>
   </organization>
 
+  <properties>
+    <top.dir>${project.basedir}/..</top.dir>
+  </properties>
+
   <build>
     <plugins>
       <plugin>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/29ea5035/phoenix-spark/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-spark/pom.xml b/phoenix-spark/pom.xml
index e42a7ec..1747573 100644
--- a/phoenix-spark/pom.xml
+++ b/phoenix-spark/pom.xml
@@ -37,6 +37,7 @@
     <spark.version>1.3.0</spark.version>
     <scala.version>2.10.4</scala.version>
     <scala.binary.version>2.10</scala.binary.version>
+    <top.dir>${project.basedir}/..</top.dir>
   </properties>
 
   <dependencies>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/29ea5035/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index f6708cc..be44399 100644
--- a/pom.xml
+++ b/pom.xml
@@ -76,6 +76,7 @@
     <antlr-input.dir>src/main/antlr3</antlr-input.dir>
     <antlr-output.dir>target/generated-sources/antlr3</antlr-output.dir>
     <test.output.tofile>true</test.output.tofile>
+    <top.dir>${project.basedir}</top.dir>
 
     <!-- Hadoop Versions -->
     <hbase.version>1.1.0</hbase.version>
@@ -332,6 +333,28 @@
     <plugins>
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-checkstyle-plugin</artifactId>
+        <version>2.13</version>
+        <executions>
+          <execution>
+            <id>validate</id>
+            <phase>validate</phase>
+            <configuration>
+              <configLocation>${top.dir}/src/main/config/checkstyle/checker.xml</configLocation>
+              <suppressionsLocation>${top.dir}/src/main/config/checkstyle/suppressions.xml</suppressionsLocation>
+              <consoleOutput>true</consoleOutput>
+              <headerLocation>${top.dir}/src/main/config/checkstyle/header.txt</headerLocation>
+              <failOnViolation><!--true-->false</failOnViolation>
+              <includeTestSourceDirectory><!--true-->false</includeTestSourceDirectory>
+            </configuration>
+            <goals>
+              <goal>check</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-source-plugin</artifactId>
         <version>2.2.1</version>
         <executions>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/29ea5035/src/main/config/checkstyle/checker.xml
----------------------------------------------------------------------
diff --git a/src/main/config/checkstyle/checker.xml b/src/main/config/checkstyle/checker.xml
new file mode 100644
index 0000000..ecf3946
--- /dev/null
+++ b/src/main/config/checkstyle/checker.xml
@@ -0,0 +1,281 @@
+<?xml version="1.0"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to you under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+-->
+
+<!--
+  This version of checkstyle is based on the Apache Calcite checkstyle
+  checkstyle configuration, which in turn is based on Giraph and Hadoop and
+  common-math configurations.
+
+  The documentation for checkstyle is available at
+
+  http://checkstyle.sourceforge.net
+-->
+
+<!DOCTYPE module PUBLIC "-//Puppy Crawl//DTD Check Configuration 1.1//EN" "http://www.puppycrawl.com/dtds/configuration_1_1.dtd">
+
+<!-- Calcite customization of default Checkstyle behavior -->
+<module name="Checker">
+  <property name="localeLanguage" value="en"/>
+
+  <!-- Checks for headers -->
+  <!-- See http://checkstyle.sf.net/config_header.html -->
+    <!-- Verify that EVERY source file has the appropriate license -->
+  <module name="Header">
+    <property name="headerFile" value="${checkstyle.header.file}"/>
+  </module>
+
+  <!-- Checks for Javadoc comments (checker).           -->
+  <!-- See http://checkstyle.sf.net/config_javadoc.html -->
+    <!-- Require package javadoc -->
+  <module name="JavadocPackage"/>
+
+  <!-- Miscellaneous other checks (checker).         -->
+  <!-- See http://checkstyle.sf.net/config_misc.html -->
+    <!-- Require files to end with newline characters -->
+  <module name="NewlineAtEndOfFile">
+    <property name="lineSeparator" value="lf"/>
+  </module>
+
+  <!-- Checks for whitespace (tree walker)                 -->
+  <!-- See http://checkstyle.sf.net/config_whitespace.html -->
+    <!-- No tabs allowed! -->
+  <module name="FileTabCharacter"/>
+
+  <module name="TreeWalker">
+    <property name="cacheFile" value="target/checkstyle-cachefile"/>
+
+    <!-- Checks for blocks. You know, those {}'s         -->
+    <!-- See http://checkstyle.sf.net/config_blocks.html -->
+      <!-- No empty blocks (i.e. catch); must contain at least a comment -->
+    <module name="EmptyBlock">
+      <property name="option" value="text"/>
+    </module>
+    <module name="AvoidNestedBlocks">
+        <property name="allowInSwitchCase" value="true"/>
+    </module>
+    <module name="LeftCurly"/>
+      <!-- No if/else/do/for/while without braces -->
+    <module name="NeedBraces"/>
+    <module name="RightCurly"/>
+
+    <!-- Checks for class design                         -->
+    <!-- See http://checkstyle.sf.net/config_design.html -->
+      <!-- Utility class should not be instantiated, they must have a
+	   private constructor -->
+    <module name="HideUtilityClassConstructor"/>
+
+    <!-- Checks for common coding problems               -->
+    <!-- See http://checkstyle.sf.net/config_coding.html -->
+    <module name="EmptyStatement"/>
+      <!-- Require hash code override when equals is -->
+    <module name="EqualsHashCode"/>
+      <!-- Disallow unnecessary instantiation of Boolean, String -->
+    <module name="IllegalInstantiation">
+      <property name="classes" value="java.lang.Boolean, java.lang.String"/>
+    </module>
+      <!-- Switch statements should be complete and with independent cases -->
+    <module name="FallThrough"/>
+    <!-- For hadoop_yarn profile, some YARN exceptions aren't loading in checkstyle -->
+    <module name="RedundantThrows">
+        <property name="suppressLoadErrors" value="true" />
+    </module>
+    <module name="SimplifyBooleanExpression"/>
+    <module name="SimplifyBooleanReturn"/>
+      <!-- Only one statement per line allowed -->
+    <module name="OneStatementPerLine"/>
+      <!-- Don't add up parentheses when they are not required -->
+    <module name="UnnecessaryParentheses" />
+      <!-- Don't use = or != for string comparisons -->
+    <module name="StringLiteralEquality" />
+      <!-- Don't declare multiple variables in the same statement -->
+    <module name="MultipleVariableDeclarations" />
+      <!-- String literals more than one character long should not be
+	   repeated several times -->
+      <!-- the "unchecked" string is also accepted to allow
+	   @SuppressWarnings("unchecked") -->
+      <!-- Disabling for now until we have a better ignoreStringsRegexp -->
+      <!--
+    <module name="MultipleStringLiterals" >
+      <property name="ignoreStringsRegexp" value='^(("")|(".")|("unchecked"))$'/>
+    </module>
+      -->
+
+    <!-- Checks for imports                              -->
+    <!-- See http://checkstyle.sf.net/config_import.html -->
+    <module name="RedundantImport"/>
+      <!-- Import should be explicit, and only from pure java packages.
+           But we allow imports that are only used in javadoc. -->
+    <module name="UnusedImports">
+      <property name="processJavadoc" value="true"/>
+    </module>
+    <module name="IllegalImport" />
+    <module name="AvoidStarImport" />
+    <module name="ImportOrder">
+      <property name="groups" value="java,javax,lib,shared,common,platform,org,com,io,net,scala,clover"/>
+      <property name="ordered" value="true"/>
+      <property name="separated" value="true"/>
+      <property name="option" value="bottom"/>
+    </module>
+
+    <!-- Checks for Javadoc comments (tree walker).       -->
+    <!-- See http://checkstyle.sf.net/config_javadoc.html -->
+      <!-- Javadoc must be formatted correctly -->
+    <module name="JavadocStyle">
+      <property name="checkFirstSentence" value="false"/>
+    </module>
+      <!-- Must have class / interface header comments -->
+    <module name="JavadocType"/>
+
+    <!-- Miscellaneous other checks (tree walker).     -->
+    <!-- See http://checkstyle.sf.net/config_misc.html -->
+      <!-- Java style arrays -->
+    <module name="ArrayTypeStyle"/>
+      <!-- Indentation -->
+    <module name="Indentation">
+      <property name="caseIndent" value="0"/>
+      <property name="basicOffset" value="4"/>
+      <property name="braceAdjustment" value="0"/>
+    </module>
+      <!-- Turn this on to see what needs to be done
+    <module name="TodoComment"/>
+       -->
+    <module name="UpperEll"/>
+
+    <module name="OperatorWrap"/>
+
+    <!-- Modifier Checks                                    -->
+    <!-- See http://checkstyle.sf.net/config_modifiers.html -->
+      <!-- Use a consistent way to put modifiers -->
+    <module name="ModifierOrder"/>
+    <module name="RedundantModifier"/>
+
+    <!-- Checks for Naming Conventions.                  -->
+    <!-- See http://checkstyle.sf.net/config_naming.html -->
+      <!-- Constant names should obey the traditional all uppercase
+	   naming convention -->
+    <module name="ConstantName"/>
+    <module name="LocalFinalVariableName">
+      <!-- Allow '_' except first. -->
+      <property name="format" value="^[a-z][a-zA-Z0-9_]*$"/>
+    </module>
+    <module name="LocalVariableName">
+      <!-- Allow '_' except first. -->
+      <property name="format" value="^[a-z][a-zA-Z0-9_]*$"/>
+    </module>
+    <module name="MemberName"/>
+    <module name="MethodName">
+      <!-- Allow trailing '_', signifying private methods.
+           Also allow '_' prefix, indicating disabled method or junit test. -->
+      <property name="format" value="^_?[a-z][a-zA-Z0-9]*_?$"/>
+    </module>
+    <module name="PackageName"/>
+    <module name="ParameterName">
+      <!-- Allow trailing '_'. -->
+      <property name="format" value="^[a-z][a-zA-Z0-9]*_?$"/>
+    </module>
+    <module name="StaticVariableName"/>
+    <module name="TypeName"/>
+
+    <!-- Checks for regexp expressions.                  -->
+    <!-- See http://checkstyle.sf.net/config_regexp.html -->
+
+    <!-- No trailing whitespace -->
+    <module name="Regexp">
+      <property name="format" value="[ \t]+$"/>
+      <property name="illegalPattern" value="true"/>
+      <property name="message" value="Trailing whitespace"/>
+    </module>
+
+    <!-- Authors should be in pom.xml file -->
+    <module name="Regexp">
+      <property name="format" value="@author"/>
+      <property name="illegalPattern" value="true"/>
+      <property name="message" value="developers names should be in pom file"/>
+    </module>
+
+    <!-- No multi-line C-style comments except at start of line. -->
+    <module name="Regexp">
+      <property name="format" value="^ +/\*[^*][^/]$"/>
+      <property name="illegalPattern" value="true"/>
+      <property name="message" value="C-style comment"/>
+    </module>
+
+    <module name="Regexp">
+      <property name="format" value="^ +/\*$"/>
+      <property name="illegalPattern" value="true"/>
+      <property name="message" value="C-style comment"/>
+    </module>
+
+    <!-- Checks for Size Violations.                    -->
+    <!-- See http://checkstyle.sf.net/config_sizes.html -->
+    <!-- Lines cannot exceed 100 chars, except if they are hyperlinks
+         or strings (possibly preceded by '+' and followed by say '),'. -->
+    <module name="LineLength">
+      <property name="max" value="100"/>
+      <property name="ignorePattern" value="^import|@see|@link|@BaseMessage|href|^[ +]*&quot;.*&quot;[);,]*$"/>
+    </module>
+      <!-- Over time, we will revise this down -->
+    <module name="MethodLength">
+      <property name="max" value="390"/>
+    </module>
+
+    <!-- Checks for whitespace (tree walker)                 -->
+    <!-- See http://checkstyle.sf.net/config_whitespace.html -->
+    <module name="EmptyForIteratorPad"/>
+      <!-- Spacing around methods -->
+    <module name="MethodParamPad">
+      <property name="option" value="nospace"/>
+      <property name="allowLineBreaks" value="true"/>
+     </module>
+      <!-- No whitespace before a token -->
+    <module name="NoWhitespaceBefore"/>
+      <!-- Whitespace after tokens is required -->
+    <module name="WhitespaceAfter"/>
+      <!-- Whitespace around tokens is required -->
+    <module name="WhitespaceAround">
+        <property name="allowEmptyConstructors" value="true"/>
+        <property name="allowEmptyMethods" value="true"/>
+    </module>
+    <module name="ParenPad"/>
+    <module name="TypecastParenPad"/>
+      <!-- No extra whitespace around types -->
+    <module name="GenericWhitespace"/>
+
+    <!-- Required for SuppressionCommentFilter below -->
+    <module name="FileContentsHolder"/>
+  </module>
+
+  <!-- Setup special comments to suppress specific checks from source files -->
+  <module name="SuppressionCommentFilter">
+    <property name="offCommentFormat" value="CHECKSTYLE\: stop ([\w\|]+)"/>
+    <property name="onCommentFormat"  value="CHECKSTYLE\: resume ([\w\|]+)"/>
+    <property name="checkFormat"      value="$1"/>
+  </module>
+
+  <!-- Turn off all checks between OFF and ON -->
+  <module name="SuppressionCommentFilter">
+    <property name="offCommentFormat" value="CHECKSTYLE\: OFF"/>
+    <property name="onCommentFormat"  value="CHECKSTYLE\: ON"/>
+  </module>
+
+  <!-- Turn off checks for the next N lines. -->
+  <module name="SuppressWithNearbyCommentFilter">
+    <property name="commentFormat" value="CHECKSTYLE: +IGNORE (\d+)"/>
+    <property name="influenceFormat" value="$1"/>
+  </module>
+</module>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/29ea5035/src/main/config/checkstyle/header.txt
----------------------------------------------------------------------
diff --git a/src/main/config/checkstyle/header.txt b/src/main/config/checkstyle/header.txt
new file mode 100644
index 0000000..2a42971
--- /dev/null
+++ b/src/main/config/checkstyle/header.txt
@@ -0,0 +1,16 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */

http://git-wip-us.apache.org/repos/asf/phoenix/blob/29ea5035/src/main/config/checkstyle/suppressions.xml
----------------------------------------------------------------------
diff --git a/src/main/config/checkstyle/suppressions.xml b/src/main/config/checkstyle/suppressions.xml
new file mode 100644
index 0000000..6662eca
--- /dev/null
+++ b/src/main/config/checkstyle/suppressions.xml
@@ -0,0 +1,46 @@
+<?xml version="1.0"?>
+<!DOCTYPE suppressions PUBLIC
+        "-//Puppy Crawl//DTD Suppressions 1.1//EN"
+        "http://www.puppycrawl.com/dtds/suppressions_1_1.dtd">
+<!--
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to you under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+-->
+<suppressions>
+  <!-- Suppress checks on generated files. -->
+  <suppress checks="Header" files="CalciteResource.properties"/>
+  <suppress checks=".*" files="org-apache-calcite-jdbc.properties"/>
+  <suppress checks=".*" files="Foo.java"/>
+  <suppress checks=".*" files=".*/target/maven-archiver/pom.properties"/>
+  <suppress checks=".*" files="git.properties"/>
+  <suppress checks=".*" files="trace.properties"/>
+  <suppress checks=".*" files="release.properties"/>
+
+  <!-- This file triggers https://github.com/checkstyle/checkstyle/issues/92,
+       through no fault of its own. -->
+  <suppress checks=".*" files="SqlSimpleParser.java"/>
+
+  <!-- Don't complain about field names such as cust_id -->
+  <suppress checks=".*Name" files="JdbcExample.java"/>
+
+  <!-- Suppress JavadocPackage in the test packages -->
+  <suppress checks="JavadocPackage" files="src[/\\]test[/\\]java[/\\]"/>
+
+  <!-- And likewise in ubenchmark -->
+  <suppress checks="JavadocPackage" files="StatementTest.java"/>
+
+  <!-- Method names in Resource can have underscores -->
+  <suppress checks="MethodName" files="CalciteResource.java"/>
+</suppressions>


[14/31] phoenix git commit: PHOENIX-2022 Make BaseRegionScanner.next abstract

Posted by nd...@apache.org.
PHOENIX-2022 Make BaseRegionScanner.next abstract

Avoid infinite recursion by removing a recursive call within
BaseRegionScanner.next, which was already being used as an
abstract method.


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

Branch: refs/heads/4.x-HBase-1.1
Commit: c1882ee279293b560fda9beb10ac50b8d3ead589
Parents: b2c0cb9
Author: Gabriel Reid <ga...@ngdata.com>
Authored: Mon Jun 1 08:57:22 2015 +0200
Committer: Gabriel Reid <ga...@ngdata.com>
Committed: Mon Jun 1 17:22:49 2015 +0200

----------------------------------------------------------------------
 .../java/org/apache/phoenix/coprocessor/BaseRegionScanner.java   | 4 +---
 1 file changed, 1 insertion(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/c1882ee2/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseRegionScanner.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseRegionScanner.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseRegionScanner.java
index 828f776..3f73048 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseRegionScanner.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseRegionScanner.java
@@ -33,9 +33,7 @@ public abstract class BaseRegionScanner implements RegionScanner {
     }
 
     @Override
-    public boolean next(List<Cell> results) throws IOException {
-        return next(results);
-    }
+    public abstract boolean next(List<Cell> results) throws IOException;
 
     @Override
     public boolean next(List<Cell> result, ScannerContext scannerContext) throws IOException {


[26/31] phoenix git commit: PHOENIX-2032 psql.py is broken after PHOENIX-2013

Posted by nd...@apache.org.
PHOENIX-2032 psql.py is broken after PHOENIX-2013


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

Branch: refs/heads/4.x-HBase-1.1
Commit: d0bcb7b2304133031b945d50e01f0f1d5fd023d4
Parents: e64f61b
Author: Nick Dimiduk <nd...@apache.org>
Authored: Fri Jun 12 10:23:05 2015 -0700
Committer: Nick Dimiduk <nd...@apache.org>
Committed: Fri Jun 12 17:01:27 2015 -0700

----------------------------------------------------------------------
 phoenix-assembly/pom.xml              |  4 ----
 phoenix-assembly/src/build/client.xml | 27 +++++++++++++++++++++++----
 2 files changed, 23 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/d0bcb7b2/phoenix-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-assembly/pom.xml b/phoenix-assembly/pom.xml
index d275d03..ebc5d71 100644
--- a/phoenix-assembly/pom.xml
+++ b/phoenix-assembly/pom.xml
@@ -152,10 +152,6 @@
     </dependency>
     <dependency>
       <groupId>org.apache.phoenix</groupId>
-      <artifactId>phoenix-spark</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.phoenix</groupId>
       <artifactId>phoenix-server</artifactId>
     </dependency>
     <dependency>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d0bcb7b2/phoenix-assembly/src/build/client.xml
----------------------------------------------------------------------
diff --git a/phoenix-assembly/src/build/client.xml b/phoenix-assembly/src/build/client.xml
index 101ccd6..e99bb19 100644
--- a/phoenix-assembly/src/build/client.xml
+++ b/phoenix-assembly/src/build/client.xml
@@ -53,13 +53,32 @@
     </dependencySet>
 
     <!-- Make sure we get all the components, not just the minimal client ones (e.g.
-      phoenix-flume, phoenix-pig, etc) -->
+      phoenix-flume, phoenix-pig, etc). We should exclude phoenix-server and
+      phoenix-server-client in the future, see PHOENIX-2032, PHOENIX-2038 -->
     <dependencySet>
       <outputDirectory>/</outputDirectory>
       <unpack>true</unpack>
-      <includes>
-        <include>org.apache.phoenix:phoenix-*</include>
-      </includes>
+      <!-- multiple deps provide some variant of LICENSE files/directories. These
+           overwrite each other at best, at worst conflict on case-insensitive
+           filesystems like HDFS+ and FAT32. Just exclude them -->
+      <unpackOptions>
+        <excludes>
+          <exclude>*license*</exclude>
+          <exclude>*LICENSE*</exclude>
+          <exclude>**/license/**</exclude>
+          <exclude>**/LICENSE/**</exclude>
+        </excludes>
+      </unpackOptions>
+      <!-- this is default, but make intentions clear -->
+      <useTransitiveDependencies>true</useTransitiveDependencies>
+      <!-- "When <include> subelements are present, they define a set of
+           artifact coordinates to include. If none is present, then <includes>
+           represents all valid values"
+           https://maven.apache.org/plugins/maven-assembly-plugin/assembly.html#class_dependencySet
+           This means bring in all dependencies transitively of the
+           phoenix-assembly module.
+      -->
+      <includes />
     </dependencySet>
   </dependencySets>
 </assembly>


[13/31] phoenix git commit: PHOENIX-2010 Properly validate number of arguments passed to the functions in FunctionParseNode#validate(Rajeshbabu)

Posted by nd...@apache.org.
PHOENIX-2010 Properly validate number of arguments passed to the functions in FunctionParseNode#validate(Rajeshbabu)


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

Branch: refs/heads/4.x-HBase-1.1
Commit: b2c0cb9002ee881f21d968817c386a98d39074ca
Parents: a600cc4
Author: Rajeshbabu Chintaguntla <ra...@apache.org>
Authored: Sun May 31 07:40:39 2015 +0530
Committer: Rajeshbabu Chintaguntla <ra...@apache.org>
Committed: Sun May 31 07:40:39 2015 +0530

----------------------------------------------------------------------
 .../phoenix/end2end/UserDefinedFunctionsIT.java       | 14 ++++++++++++++
 .../org/apache/phoenix/parse/FunctionParseNode.java   |  4 ++++
 .../main/java/org/apache/phoenix/parse/PFunction.java |  4 +---
 3 files changed, 19 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/b2c0cb90/phoenix-core/src/it/java/org/apache/phoenix/end2end/UserDefinedFunctionsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UserDefinedFunctionsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UserDefinedFunctionsIT.java
index 7dbde3c..868e19d 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UserDefinedFunctionsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UserDefinedFunctionsIT.java
@@ -442,6 +442,20 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT{
         rs = stmt.executeQuery("select k from t9 where mysum9(k)=11");
         assertTrue(rs.next());
         assertEquals(1, rs.getInt(1));
+        try {
+            rs = stmt.executeQuery("select k from t9 where mysum9(k,10,'x')=11");
+            fail("FunctionNotFoundException should be thrown");
+        } catch(FunctionNotFoundException e) {
+        } catch(Exception e) {
+            fail("FunctionNotFoundException should be thrown");
+        }
+        try {
+            rs = stmt.executeQuery("select mysum9() from t9");
+            fail("FunctionNotFoundException should be thrown");
+        } catch(FunctionNotFoundException e) {
+        } catch(Exception e) {
+            fail("FunctionNotFoundException should be thrown");
+        }
         stmt.execute("drop function mysum9");
         try {
             rs = stmt.executeQuery("select k from t9 where mysum9(k)=11");

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b2c0cb90/phoenix-core/src/main/java/org/apache/phoenix/parse/FunctionParseNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/FunctionParseNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/FunctionParseNode.java
index d1001ee..be52d89 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/FunctionParseNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/FunctionParseNode.java
@@ -41,6 +41,7 @@ import org.apache.phoenix.expression.function.FunctionExpression;
 import org.apache.phoenix.expression.function.UDFExpression;
 import org.apache.phoenix.parse.PFunction.FunctionArgument;
 import org.apache.phoenix.schema.ArgumentTypeMismatchException;
+import org.apache.phoenix.schema.FunctionNotFoundException;
 import org.apache.phoenix.schema.ValueRangeExcpetion;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PDataTypeFactory;
@@ -133,6 +134,9 @@ public class FunctionParseNode extends CompoundParseNode {
     public List<Expression> validate(List<Expression> children, StatementContext context) throws SQLException {
         BuiltInFunctionInfo info = this.getInfo();
         BuiltInFunctionArgInfo[] args = info.getArgs();
+        if (args.length < children.size() || info.getRequiredArgCount() > children.size()) {
+            throw new FunctionNotFoundException(this.name);
+        }
         if (args.length > children.size()) {
             List<Expression> moreChildren = new ArrayList<Expression>(children);
             for (int i = children.size(); i < info.getArgs().length; i++) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b2c0cb90/phoenix-core/src/main/java/org/apache/phoenix/parse/PFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/PFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/PFunction.java
index 351bec7..aeed3ac 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/PFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/PFunction.java
@@ -96,9 +96,7 @@ public class PFunction implements PMetaDataEntity {
     }
 
     public PFunction(PFunction function) {
-        this(function.getTenantId(), function.getFunctionName(), function.getFunctionArguments(),
-                function.getReturnType(), function.getClassName(), function.getJarPath(), function
-                        .getTimeStamp());
+        this(function, function.isTemporaryFunction());
     }
 
     public String getFunctionName() {


[22/31] phoenix git commit: PHOENIX-1978 UDF ArgumentTypeMismatchException(Rajeshbabu)

Posted by nd...@apache.org.
PHOENIX-1978 UDF ArgumentTypeMismatchException(Rajeshbabu)


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

Branch: refs/heads/4.x-HBase-1.1
Commit: 18b9e72756642e127b2e227ea46a4f70401e6187
Parents: 58ee706
Author: Rajeshbabu Chintaguntla <ra...@apache.org>
Authored: Fri Jun 5 09:04:17 2015 +0530
Committer: Rajeshbabu Chintaguntla <ra...@apache.org>
Committed: Fri Jun 5 09:04:17 2015 +0530

----------------------------------------------------------------------
 .../phoenix/end2end/UserDefinedFunctionsIT.java | 58 ++++++++++++++++++--
 phoenix-core/src/main/antlr3/PhoenixSQL.g       | 17 +++---
 2 files changed, 61 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/18b9e727/phoenix-core/src/it/java/org/apache/phoenix/end2end/UserDefinedFunctionsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UserDefinedFunctionsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UserDefinedFunctionsIT.java
index 868e19d..c6bd62f 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UserDefinedFunctionsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UserDefinedFunctionsIT.java
@@ -58,6 +58,8 @@ import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.FunctionAlreadyExistsException;
 import org.apache.phoenix.schema.FunctionNotFoundException;
 import org.apache.phoenix.schema.ValueRangeExcpetion;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PArrayDataType;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
@@ -121,11 +123,31 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT{
                     .append("        ptr.set(PInteger.INSTANCE.toBytes((Integer)sum));\n")
                     .append("        return true;\n")
                     .append("    }\n").toString();
-
+    private static String ARRAY_INDEX_EVALUATE_METHOD =
+            new StringBuffer()
+                    .append("    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {\n")
+                    .append("        Expression indexExpr = children.get(1);\n")
+                    .append("        if (!indexExpr.evaluate(tuple, ptr)) {\n")
+                    .append("           return false;\n")
+                    .append("        } else if (ptr.getLength() == 0) {\n")
+                    .append("           return true;\n")
+                    .append("        }\n")
+                    .append("        // Use Codec to prevent Integer object allocation\n")
+                    .append("        int index = PInteger.INSTANCE.getCodec().decodeInt(ptr, indexExpr.getSortOrder());\n")
+                    .append("        if(index < 0) {\n")
+                    .append("           throw new ParseException(\"Index cannot be negative :\" + index);\n")
+                    .append("        }\n")
+                    .append("        Expression arrayExpr = children.get(0);\n")
+                    .append("        return PArrayDataType.positionAtArrayElement(tuple, ptr, index, arrayExpr, getDataType(),getMaxLength());\n")
+                    .append("    }\n").toString();
+    
+    
     private static String MY_REVERSE_CLASS_NAME = "MyReverse";
     private static String MY_SUM_CLASS_NAME = "MySum";
-    private static String MY_REVERSE_PROGRAM = getProgram(MY_REVERSE_CLASS_NAME, STRING_REVERSE_EVALUATE_METHOD, "PVarchar");
-    private static String MY_SUM_PROGRAM = getProgram(MY_SUM_CLASS_NAME, SUM_COLUMN_VALUES_EVALUATE_METHOD, "PInteger");
+    private static String MY_ARRAY_INDEX_CLASS_NAME = "MyArrayIndex";
+    private static String MY_REVERSE_PROGRAM = getProgram(MY_REVERSE_CLASS_NAME, STRING_REVERSE_EVALUATE_METHOD, "return PVarchar.INSTANCE;");
+    private static String MY_SUM_PROGRAM = getProgram(MY_SUM_CLASS_NAME, SUM_COLUMN_VALUES_EVALUATE_METHOD, "return PInteger.INSTANCE;");
+    private static String MY_ARRAY_INDEX_PROGRAM = getProgram(MY_ARRAY_INDEX_CLASS_NAME, ARRAY_INDEX_EVALUATE_METHOD, "return PDataType.fromTypeId(children.get(0).getDataType().getSqlType()- PDataType.ARRAY_TYPE_BASE);");
     private static Properties EMPTY_PROPS = new Properties();
     
 
@@ -144,6 +166,8 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT{
                 .append("import org.apache.phoenix.schema.types.PInteger;\n")
                 .append("import org.apache.phoenix.schema.types.PVarchar;\n")
                 .append("import org.apache.phoenix.util.StringUtil;\n")
+                .append("import org.apache.phoenix.schema.types.PArrayDataType;\n")
+                .append("import org.apache.phoenix.parse.ParseException;\n")
                 .append("public class "+className+" extends ScalarFunction{\n")
                 .append("    public static final String NAME = \"MY_REVERSE\";\n")
                 .append("    public "+className+"() {\n")
@@ -159,7 +183,7 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT{
                 .append("    }\n")
                 .append("  @Override\n")
                 .append("   public PDataType getDataType() {\n")
-                .append("       return "+returnType+".INSTANCE;\n")
+                .append(returnType+"\n")
                 .append("    }\n")
                 .append("    @Override\n")
                 .append("    public String getName() {\n")
@@ -181,6 +205,8 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT{
         UDFExpression.setConfig(conf);
         compileTestClass(MY_REVERSE_CLASS_NAME, MY_REVERSE_PROGRAM, 1);
         compileTestClass(MY_SUM_CLASS_NAME, MY_SUM_PROGRAM, 2);
+        compileTestClass(MY_ARRAY_INDEX_CLASS_NAME, MY_ARRAY_INDEX_PROGRAM, 3);
+
         String clientPort = util.getConfiguration().get(QueryServices.ZOOKEEPER_PORT_ATTRIB);
         url =
                 JDBC_PROTOCOL + JDBC_PROTOCOL_SEPARATOR + LOCALHOST + JDBC_PROTOCOL_SEPARATOR
@@ -265,6 +291,28 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT{
         } catch(FunctionNotFoundException e) {
             
         }
+        conn.createStatement().execute("CREATE TABLE TESTTABLE10(ID VARCHAR NOT NULL, NAME VARCHAR ARRAY, CITY VARCHAR ARRAY CONSTRAINT pk PRIMARY KEY (ID) )");
+        conn.createStatement().execute("create function UDF_ARRAY_ELEM(VARCHAR ARRAY, INTEGER) returns VARCHAR as 'org.apache.phoenix.end2end."+MY_ARRAY_INDEX_CLASS_NAME+"' using jar "
+                + "'"+util.getConfiguration().get(DYNAMIC_JARS_DIR_KEY) + "/myjar3.jar"+"'");
+        conn.createStatement().execute("UPSERT INTO TESTTABLE10(ID,NAME,CITY) VALUES('111', ARRAY['JOHN','MIKE','BOB'], ARRAY['NYC','LA','SF'])");
+        conn.createStatement().execute("UPSERT INTO TESTTABLE10(ID,NAME,CITY) VALUES('112', ARRAY['CHEN','CARL','ALICE'], ARRAY['BOSTON','WASHINGTON','PALO ALTO'])");
+        conn.commit();
+        rs = conn.createStatement().executeQuery("SELECT ID, UDF_ARRAY_ELEM(NAME, 2) FROM TESTTABLE10");
+        assertTrue(rs.next());
+        assertEquals("111", rs.getString(1));
+        assertEquals("MIKE", rs.getString(2));
+        assertTrue(rs.next());
+        assertEquals("112", rs.getString(1));
+        assertEquals("CARL", rs.getString(2));
+        assertFalse(rs.next());
+        rs = conn2.createStatement().executeQuery("SELECT ID, UDF_ARRAY_ELEM(NAME, 2) FROM TESTTABLE10");
+        assertTrue(rs.next());
+        assertEquals("111", rs.getString(1));
+        assertEquals("MIKE", rs.getString(2));
+        assertTrue(rs.next());
+        assertEquals("112", rs.getString(1));
+        assertEquals("CARL", rs.getString(2));
+        assertFalse(rs.next());
     }
 
     @Test
@@ -326,7 +374,7 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT{
         rs = tenant2Conn.createStatement().executeQuery("select * from t7 where MYFUNCTION(k1)=12");
         assertTrue(rs.next());
         assertEquals(1, rs.getInt(1));
-        assertEquals(2, rs.getInt(2));        
+        assertEquals(2, rs.getInt(2)); 
         assertEquals("jock", rs.getString(3));
         assertFalse(rs.next());
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/18b9e727/phoenix-core/src/main/antlr3/PhoenixSQL.g
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/antlr3/PhoenixSQL.g b/phoenix-core/src/main/antlr3/PhoenixSQL.g
index d2bb241..4f7cb34 100644
--- a/phoenix-core/src/main/antlr3/PhoenixSQL.g
+++ b/phoenix-core/src/main/antlr3/PhoenixSQL.g
@@ -917,17 +917,16 @@ zero_or_more_expressions returns [List<ParseNode> ret]
 
 zero_or_more_data_types returns [List<FunctionArgument> ret]
 @init{ret = new ArrayList<FunctionArgument>(); }
-    : (dt = identifier (LPAREN l=NUMBER (COMMA s=NUMBER)? RPAREN)? ar=ARRAY? (lsq=LSQUARE (a=NUMBER)? RSQUARE)? (c = CONSTANT)? (DEFAULTVALUE EQ dv = value_expression)? (MINVALUE EQ minv = value_expression)?  (MAXVALUE EQ maxv = value_expression)? 
-    {$ret.add(new FunctionArgument(dt,  ar != null || lsq != null, c!=null, 
-    dv == null ? null : LiteralExpression.newConstant(((LiteralParseNode)dv).getValue()), 
-    minv == null ? null : LiteralExpression.newConstant(((LiteralParseNode)minv).getValue()), 
-    maxv == null ? null : LiteralExpression.newConstant(((LiteralParseNode)maxv).getValue())));})? (COMMA (dt = identifier (LPAREN l=NUMBER (COMMA s=NUMBER)? RPAREN)? ar=ARRAY? (lsq=LSQUARE (a=NUMBER)? RSQUARE)? (c = CONSTANT)? (DEFAULTVALUE EQ dv = value_expression)? (MINVALUE EQ minv = value_expression)?  (MAXVALUE EQ maxv = value_expression)?
-    {$ret.add(new FunctionArgument(dt,  ar != null || lsq != null, c!=null, 
+    : (fa = function_argument {$ret.add(fa);})? (COMMA fa = function_argument {$ret.add(fa);})* 
+	;
+
+function_argument returns [FunctionArgument ret]
+	: (dt = identifier (LPAREN l=NUMBER (COMMA s=NUMBER)? RPAREN)? ar=ARRAY? (lsq=LSQUARE (a=NUMBER)? RSQUARE)? (c = CONSTANT)? (DEFAULTVALUE EQ dv = value_expression)? (MINVALUE EQ minv = value_expression)?  (MAXVALUE EQ maxv = value_expression)? 
+	{ $ret = new FunctionArgument(dt,  ar != null || lsq != null, c!=null, 
     dv == null ? null : LiteralExpression.newConstant(((LiteralParseNode)dv).getValue()), 
     minv == null ? null : LiteralExpression.newConstant(((LiteralParseNode)minv).getValue()), 
-    maxv == null ? null : LiteralExpression.newConstant(((LiteralParseNode)maxv).getValue())));} ))*
-;
-
+    maxv == null ? null : LiteralExpression.newConstant(((LiteralParseNode)maxv).getValue()));})
+	;
 value_expression_list returns [List<ParseNode> ret]
 @init{ret = new ArrayList<ParseNode>(); }
     :  LPAREN e = value_expression {$ret.add(e);}  (COMMA e = value_expression {$ret.add(e);} )* RPAREN


[27/31] phoenix git commit: PHOENIX-2040 Mark spark/scala dependencies as 'provided' (Josh Mahonin)

Posted by nd...@apache.org.
PHOENIX-2040 Mark spark/scala dependencies as 'provided' (Josh Mahonin)


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

Branch: refs/heads/4.x-HBase-1.1
Commit: 43c722ca6d2d55347d1f2caf7641ce03339e1e1e
Parents: d0bcb7b
Author: Nick Dimiduk <nd...@apache.org>
Authored: Mon Jun 15 16:16:03 2015 -0700
Committer: Nick Dimiduk <nd...@apache.org>
Committed: Mon Jun 15 16:16:30 2015 -0700

----------------------------------------------------------------------
 phoenix-assembly/pom.xml |  4 ++++
 phoenix-spark/pom.xml    | 51 ++++++++++++++++++++++++-------------------
 2 files changed, 32 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/43c722ca/phoenix-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-assembly/pom.xml b/phoenix-assembly/pom.xml
index ebc5d71..d275d03 100644
--- a/phoenix-assembly/pom.xml
+++ b/phoenix-assembly/pom.xml
@@ -152,6 +152,10 @@
     </dependency>
     <dependency>
       <groupId>org.apache.phoenix</groupId>
+      <artifactId>phoenix-spark</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.phoenix</groupId>
       <artifactId>phoenix-server</artifactId>
     </dependency>
     <dependency>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/43c722ca/phoenix-spark/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-spark/pom.xml b/phoenix-spark/pom.xml
index 1747573..aea5c7e 100644
--- a/phoenix-spark/pom.xml
+++ b/phoenix-spark/pom.xml
@@ -45,12 +45,7 @@
       <groupId>org.apache.phoenix</groupId>
       <artifactId>phoenix-core</artifactId>
     </dependency>
-    <dependency>
-      <groupId>org.apache.phoenix</groupId>
-      <artifactId>phoenix-core</artifactId>
-      <classifier>tests</classifier>
-      <scope>test</scope>
-    </dependency>
+
     <!-- Force import of Spark's servlet API for unit tests -->
     <dependency>
       <groupId>javax.servlet</groupId>
@@ -59,16 +54,38 @@
       <scope>test</scope>
     </dependency>
 
+    <!-- Mark Spark / Scala as provided -->
     <dependency>
-      <groupId>junit</groupId>
-      <artifactId>junit</artifactId>
+      <groupId>org.scala-lang</groupId>
+      <artifactId>scala-library</artifactId>
+      <version>${scala.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-core_${scala.binary.version}</artifactId>
+      <version>${spark.version}</version>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-sql_${scala.binary.version}</artifactId>
+      <version>${spark.version}</version>
+      <scope>provided</scope>
+    </dependency>
+
+    <!-- Test dependencies -->
+    <dependency>
+      <groupId>org.apache.phoenix</groupId>
+      <artifactId>phoenix-core</artifactId>
+      <classifier>tests</classifier>
       <scope>test</scope>
     </dependency>
 
     <dependency>
-      <groupId>org.scala-lang</groupId>
-      <artifactId>scala-library</artifactId>
-      <version>${scala.version}</version>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
     </dependency>
 
     <dependency>
@@ -86,18 +103,6 @@
     </dependency>
 
     <dependency>
-      <groupId>org.apache.spark</groupId>
-      <artifactId>spark-core_${scala.binary.version}</artifactId>
-      <version>${spark.version}</version>
-    </dependency>
-
-    <dependency>
-      <groupId>org.apache.spark</groupId>
-      <artifactId>spark-sql_${scala.binary.version}</artifactId>
-      <version>${spark.version}</version>
-    </dependency>
-
-    <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-client</artifactId>
       <version>${hadoop-two.version}</version>


[31/31] phoenix git commit: PHOENIX-1975 Detect and use HBASE_HOME when set

Posted by nd...@apache.org.
PHOENIX-1975 Detect and use HBASE_HOME when set


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

Branch: refs/heads/4.x-HBase-1.1
Commit: 980d29c5acf785dc90ece1a7f047711e8d522a2e
Parents: 05b1b8b
Author: Nick Dimiduk <nd...@apache.org>
Authored: Wed Jun 24 13:59:00 2015 -0700
Committer: Nick Dimiduk <nd...@apache.org>
Committed: Wed Jun 24 13:59:28 2015 -0700

----------------------------------------------------------------------
 bin/phoenix_utils.py | 10 +++++++++-
 1 file changed, 9 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/980d29c5/bin/phoenix_utils.py
----------------------------------------------------------------------
diff --git a/bin/phoenix_utils.py b/bin/phoenix_utils.py
index 383e0e1..bfb4737 100755
--- a/bin/phoenix_utils.py
+++ b/bin/phoenix_utils.py
@@ -65,7 +65,15 @@ def setPath():
         phoenix_class_path = os.getenv('PHOENIX_CLASS_PATH','')
 
     global hbase_conf_dir
-    hbase_conf_dir = os.getenv('HBASE_CONF_DIR', os.getenv('HBASE_CONF_PATH', '.'))
+    # if HBASE_CONF_DIR set explicitly, use that
+    hbase_conf_dir = os.getenv('HBASE_CONF_DIR', os.getenv('HBASE_CONF_PATH'))
+    if not hbase_conf_dir:
+        # else fall back to HBASE_HOME
+        if os.getenv('HBASE_HOME'):
+            hbase_conf_dir = os.path.join(os.getenv('HBASE_HOME'), "conf")
+        else:
+            # default to pwd
+            hbase_conf_dir = '.'
     global hbase_conf_path # keep conf_path around for backward compatibility
     hbase_conf_path = hbase_conf_dir
 


[25/31] phoenix git commit: PHOENIX-2033 PQS log environment details on launch

Posted by nd...@apache.org.
PHOENIX-2033 PQS log environment details on launch


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

Branch: refs/heads/4.x-HBase-1.1
Commit: e64f61ba431b8db938bf60992bbde56f4c540946
Parents: f7d7349
Author: Nick Dimiduk <nd...@apache.org>
Authored: Tue Jun 9 17:12:21 2015 -0700
Committer: Nick Dimiduk <nd...@apache.org>
Committed: Fri Jun 12 09:38:42 2015 -0700

----------------------------------------------------------------------
 .../apache/phoenix/queryserver/server/Main.java | 69 ++++++++++++++++++++
 1 file changed, 69 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/e64f61ba/phoenix-server/src/main/java/org/apache/phoenix/queryserver/server/Main.java
----------------------------------------------------------------------
diff --git a/phoenix-server/src/main/java/org/apache/phoenix/queryserver/server/Main.java b/phoenix-server/src/main/java/org/apache/phoenix/queryserver/server/Main.java
index 55febc5..9f9bfc7 100644
--- a/phoenix-server/src/main/java/org/apache/phoenix/queryserver/server/Main.java
+++ b/phoenix-server/src/main/java/org/apache/phoenix/queryserver/server/Main.java
@@ -34,7 +34,12 @@ import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 
+import java.lang.management.ManagementFactory;
+import java.lang.management.RuntimeMXBean;
 import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 
@@ -50,6 +55,11 @@ public final class Main extends Configured implements Tool, Runnable {
       "phoenix.queryserver.http.port";
   public static final int DEFAULT_HTTP_PORT = 8765;
 
+  public static final String QUERY_SERVER_ENV_LOGGING_KEY =
+          "phoenix.queryserver.envvars.logging.disabled";
+  public static final String QUERY_SERVER_ENV_LOGGING_SKIPWORDS_KEY =
+          "phoenix.queryserver.envvars.logging.skipwords";
+
   public static final String KEYTAB_FILENAME_KEY = "phoenix.queryserver.keytab.file";
   public static final String KERBEROS_PRINCIPAL_KEY = "phoenix.queryserver.kerberos.principal";
   public static final String DNS_NAMESERVER_KEY = "phoenix.queryserver.dns.nameserver";
@@ -58,12 +68,70 @@ public final class Main extends Configured implements Tool, Runnable {
 
   protected static final Log LOG = LogFactory.getLog(Main.class);
 
+  @SuppressWarnings("serial")
+  private static final Set<String> DEFAULT_SKIP_WORDS = new HashSet<String>() {
+    {
+      add("secret");
+      add("passwd");
+      add("password");
+      add("credential");
+    }
+  };
+
   private final String[] argv;
   private final CountDownLatch runningLatch = new CountDownLatch(1);
   private HttpServer server = null;
   private int retCode = 0;
   private Throwable t = null;
 
+  /**
+   * Log information about the currently running JVM.
+   */
+  public static void logJVMInfo() {
+    // Print out vm stats before starting up.
+    RuntimeMXBean runtime = ManagementFactory.getRuntimeMXBean();
+    if (runtime != null) {
+      LOG.info("vmName=" + runtime.getVmName() + ", vmVendor=" +
+              runtime.getVmVendor() + ", vmVersion=" + runtime.getVmVersion());
+      LOG.info("vmInputArguments=" + runtime.getInputArguments());
+    }
+  }
+
+  /**
+   * Logs information about the currently running JVM process including
+   * the environment variables. Logging of env vars can be disabled by
+   * setting {@code "phoenix.envvars.logging.disabled"} to {@code "true"}.
+   * <p>If enabled, you can also exclude environment variables containing
+   * certain substrings by setting {@code "phoenix.envvars.logging.skipwords"}
+   * to comma separated list of such substrings.
+   */
+  public static void logProcessInfo(Configuration conf) {
+    // log environment variables unless asked not to
+    if (conf == null || !conf.getBoolean(QUERY_SERVER_ENV_LOGGING_KEY, false)) {
+      Set<String> skipWords = new HashSet<String>(DEFAULT_SKIP_WORDS);
+      if (conf != null) {
+        String[] confSkipWords = conf.getStrings(QUERY_SERVER_ENV_LOGGING_SKIPWORDS_KEY);
+        if (confSkipWords != null) {
+          skipWords.addAll(Arrays.asList(confSkipWords));
+        }
+      }
+
+      nextEnv:
+      for (Map.Entry<String, String> entry : System.getenv().entrySet()) {
+        String key = entry.getKey().toLowerCase();
+        String value = entry.getValue().toLowerCase();
+        // exclude variables which may contain skip words
+        for(String skipWord : skipWords) {
+          if (key.contains(skipWord) || value.contains(skipWord))
+            continue nextEnv;
+        }
+        LOG.info("env:"+entry);
+      }
+    }
+    // and JVM info
+    logJVMInfo();
+  }
+
   /** Constructor for use from {@link org.apache.hadoop.util.ToolRunner}. */
   public Main() {
     this(null, null);
@@ -112,6 +180,7 @@ public final class Main extends Configured implements Tool, Runnable {
 
   @Override
   public int run(String[] args) throws Exception {
+    logProcessInfo(getConf());
     try {
       // handle secure cluster credentials
       if ("kerberos".equalsIgnoreCase(getConf().get(HBASE_SECURITY_CONF_KEY))) {


[08/31] phoenix git commit: PHOENIX-2005 Connection utilities omit zk client port, parent znode (addendum)

Posted by nd...@apache.org.
PHOENIX-2005 Connection utilities omit zk client port, parent znode (addendum)


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

Branch: refs/heads/4.x-HBase-1.1
Commit: 5546a42226e3f0fdf0cc89f1c175ff3da7a75d8c
Parents: c6b37b9
Author: Nick Dimiduk <nd...@apache.org>
Authored: Tue May 26 17:41:04 2015 -0700
Committer: Nick Dimiduk <nd...@apache.org>
Committed: Tue May 26 17:52:24 2015 -0700

----------------------------------------------------------------------
 .../phoenix/jdbc/PhoenixEmbeddedDriver.java     |  2 +-
 .../java/org/apache/phoenix/util/QueryUtil.java |  2 +-
 .../phoenix/jdbc/PhoenixEmbeddedDriverTest.java | 20 ++++++++++++++++++++
 3 files changed, 22 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/5546a422/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixEmbeddedDriver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixEmbeddedDriver.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixEmbeddedDriver.java
index 2451603..3cfaacc 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixEmbeddedDriver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixEmbeddedDriver.java
@@ -209,7 +209,7 @@ public abstract class PhoenixEmbeddedDriver implements Driver, org.apache.phoeni
             url = url == null ? "" : url;
             url = url.startsWith(PhoenixRuntime.JDBC_PROTOCOL)
                     ? url.substring(PhoenixRuntime.JDBC_PROTOCOL.length())
-                    : url;
+                    : PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR + url;
             StringTokenizer tokenizer = new StringTokenizer(url, DELIMITERS, true);
             int nTokens = 0;
             String[] tokens = new String[5];

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5546a422/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java
index bd38983..a2d4a91 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/QueryUtil.java
@@ -290,7 +290,7 @@ public final class QueryUtil {
             throws ClassNotFoundException,
             SQLException {
         String url = getConnectionUrl(props, conf);
-        LOG.info("Creating connection with the jdbc url:" + url);
+        LOG.info("Creating connection with the jdbc url: " + url);
         PropertiesUtil.extractProperties(props, conf);
         return DriverManager.getConnection(url, props);
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5546a422/phoenix-core/src/test/java/org/apache/phoenix/jdbc/PhoenixEmbeddedDriverTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/jdbc/PhoenixEmbeddedDriverTest.java b/phoenix-core/src/test/java/org/apache/phoenix/jdbc/PhoenixEmbeddedDriverTest.java
index 083b205..4eda825 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/jdbc/PhoenixEmbeddedDriverTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/jdbc/PhoenixEmbeddedDriverTest.java
@@ -34,23 +34,33 @@ public class PhoenixEmbeddedDriverTest {
     @Test
     public void testGetConnectionInfo() throws SQLException {
         String[] urls = new String[] {
+            null,
+            "",
             "jdbc:phoenix",
             "jdbc:phoenix;test=true",
             "jdbc:phoenix:localhost",
+            "localhost",
+            "localhost;",
             "jdbc:phoenix:localhost:123",
             "jdbc:phoenix:localhost:123;foo=bar",
+            "localhost:123",
             "jdbc:phoenix:localhost:123:/hbase",
             "jdbc:phoenix:localhost:123:/foo-bar",
             "jdbc:phoenix:localhost:123:/foo-bar;foo=bas",
+            "localhost:123:/foo-bar",
             "jdbc:phoenix:localhost:/hbase",
             "jdbc:phoenix:localhost:/foo-bar",
             "jdbc:phoenix:localhost:/foo-bar;test=true",
+            "localhost:/foo-bar",
             "jdbc:phoenix:v1,v2,v3",
             "jdbc:phoenix:v1,v2,v3;",
             "jdbc:phoenix:v1,v2,v3;test=true",
+            "v1,v2,v3",
             "jdbc:phoenix:v1,v2,v3:/hbase",
             "jdbc:phoenix:v1,v2,v3:/hbase;test=true",
+            "v1,v2,v3:/foo-bar",
             "jdbc:phoenix:v1,v2,v3:123:/hbase",
+            "v1,v2,v3:123:/hbase",
             "jdbc:phoenix:v1,v2,v3:123:/hbase;test=false",
             "jdbc:phoenix:v1,v2,v3:123:/hbase:user/principal:/user.keytab;test=false",
             "jdbc:phoenix:v1,v2,v3:123:/foo-bar:user/principal:/user.keytab;test=false",
@@ -63,20 +73,30 @@ public class PhoenixEmbeddedDriverTest {
         ConnectionInfo[] infos = new ConnectionInfo[] {
             new ConnectionInfo(null,null,null),
             new ConnectionInfo(null,null,null),
+            new ConnectionInfo(null,null,null),
+            new ConnectionInfo(null,null,null),
+            new ConnectionInfo("localhost",null,null),
             new ConnectionInfo("localhost",null,null),
+            new ConnectionInfo("localhost",null,null),
+            new ConnectionInfo("localhost",123,null),
             new ConnectionInfo("localhost",123,null),
             new ConnectionInfo("localhost",123,null),
             new ConnectionInfo("localhost",123,"/hbase"),
             new ConnectionInfo("localhost",123,"/foo-bar"),
             new ConnectionInfo("localhost",123,"/foo-bar"),
+            new ConnectionInfo("localhost",123,"/foo-bar"),
             new ConnectionInfo("localhost",null,"/hbase"),
             new ConnectionInfo("localhost",null,"/foo-bar"),
             new ConnectionInfo("localhost",null,"/foo-bar"),
+            new ConnectionInfo("localhost",null,"/foo-bar"),
+            new ConnectionInfo("v1,v2,v3",null,null),
             new ConnectionInfo("v1,v2,v3",null,null),
             new ConnectionInfo("v1,v2,v3",null,null),
             new ConnectionInfo("v1,v2,v3",null,null),
             new ConnectionInfo("v1,v2,v3",null,"/hbase"),
             new ConnectionInfo("v1,v2,v3",null,"/hbase"),
+            new ConnectionInfo("v1,v2,v3",null,"/foo-bar"),
+            new ConnectionInfo("v1,v2,v3",123,"/hbase"),
             new ConnectionInfo("v1,v2,v3",123,"/hbase"),
             new ConnectionInfo("v1,v2,v3",123,"/hbase"),
             new ConnectionInfo("v1,v2,v3",123,"/hbase","user/principal", "/user.keytab" ),


[05/31] phoenix git commit: PHOENIX-2008 Integration tests are failing with HBase-1.1.0 because HBASE-13756(Rajeshbabu)

Posted by nd...@apache.org.
PHOENIX-2008 Integration tests are failing with HBase-1.1.0 because HBASE-13756(Rajeshbabu)


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

Branch: refs/heads/4.x-HBase-1.1
Commit: 56e1c0a1f348572fb73e9d0b8bbfb053df7f8710
Parents: ea622d5
Author: Rajeshbabu Chintaguntla <ra...@apache.org>
Authored: Sat May 23 23:29:31 2015 +0530
Committer: Rajeshbabu Chintaguntla <ra...@apache.org>
Committed: Sat May 23 23:29:31 2015 +0530

----------------------------------------------------------------------
 phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/56e1c0a1/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
index 54ae670..4aa28c4 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
@@ -620,6 +620,8 @@ public abstract class BaseTest {
         }
         //no point doing sanity checks when running tests.
         conf.setBoolean("hbase.table.sanity.checks", false);
+        // Remove this configuration once hbase has HBASE-13756 fix.
+        conf.set("hbase.regionserver.msginterval", "300000");
         // set the server rpc controller and rpc scheduler factory, used to configure the cluster
         conf.set(RpcControllerFactory.CUSTOM_CONTROLLER_CONF_KEY, DEFAULT_SERVER_RPC_CONTROLLER_FACTORY);
         conf.set(RSRpcServices.REGION_SERVER_RPC_SCHEDULER_FACTORY_CLASS, DEFAULT_RPC_SCHEDULER_FACTORY);


[19/31] phoenix git commit: PHOENIX-2012 RowKeyComparisonFilter logs unencoded data at DEBUG level

Posted by nd...@apache.org.
PHOENIX-2012 RowKeyComparisonFilter logs unencoded data at DEBUG level


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

Branch: refs/heads/4.x-HBase-1.1
Commit: 9c5f1111ae456f3a0934e43e02af0ef5188b9337
Parents: 29ea503
Author: Nick Dimiduk <nd...@apache.org>
Authored: Wed May 27 15:58:32 2015 -0700
Committer: Nick Dimiduk <nd...@apache.org>
Committed: Mon Jun 1 15:57:15 2015 -0700

----------------------------------------------------------------------
 .../java/org/apache/phoenix/filter/RowKeyComparisonFilter.java  | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/9c5f1111/phoenix-core/src/main/java/org/apache/phoenix/filter/RowKeyComparisonFilter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/filter/RowKeyComparisonFilter.java b/phoenix-core/src/main/java/org/apache/phoenix/filter/RowKeyComparisonFilter.java
index 2e2037b..b7de7ac 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/filter/RowKeyComparisonFilter.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/filter/RowKeyComparisonFilter.java
@@ -73,8 +73,9 @@ public class RowKeyComparisonFilter extends BooleanExpressionFilter {
         if (evaluate) {
             inputTuple.setKey(v.getRowArray(), v.getRowOffset(), v.getRowLength());
             this.keepRow = Boolean.TRUE.equals(evaluate(inputTuple));
-            if (logger.isDebugEnabled()) {
-                logger.debug("RowKeyComparisonFilter: " + (this.keepRow ? "KEEP" : "FILTER")  + " row " + inputTuple);
+            if (logger.isTraceEnabled()) {
+                logger.trace("RowKeyComparisonFilter: " + (this.keepRow ? "KEEP" : "FILTER")
+                        + " row " + inputTuple);
             }
             evaluate = false;
         }


[12/31] phoenix git commit: PHOENIX-1939 Test are failing with DoNotRetryIOException: ATABLE: null (Alicia Ying Shu)

Posted by nd...@apache.org.
PHOENIX-1939 Test are failing with DoNotRetryIOException: ATABLE: null (Alicia Ying Shu)


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

Branch: refs/heads/4.x-HBase-1.1
Commit: a600cc4d7acc2c828ae7782e59d094f99e5631f0
Parents: c95e28d
Author: Nick Dimiduk <nd...@apache.org>
Authored: Fri May 29 17:12:25 2015 -0700
Committer: Nick Dimiduk <nd...@apache.org>
Committed: Fri May 29 17:13:08 2015 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/phoenix/schema/PTableImpl.java      | 4 ++--
 .../src/test/java/org/apache/phoenix/query/BaseTest.java         | 3 ++-
 2 files changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/a600cc4d/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 bf4420c..bdc95b8 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
@@ -88,8 +88,8 @@ public class PTableImpl implements PTable {
 
     private PTableKey key;
     private PName name;
-    private PName schemaName;
-    private PName tableName;
+    private PName schemaName = PName.EMPTY_NAME;
+    private PName tableName = PName.EMPTY_NAME;
     private PName tenantId;
     private PTableType type;
     private PIndexState state;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a600cc4d/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
index 54ae670..b0574c3 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
@@ -121,7 +121,6 @@ import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.ipc.PhoenixRpcSchedulerFactory;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.ipc.controller.ServerRpcControllerFactory;
-import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.LocalIndexMerger;
 import org.apache.hadoop.hbase.regionserver.RSRpcServices;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -829,6 +828,7 @@ public abstract class BaseTest {
                     logger.info("Table " + fullTableName + " is already deleted.");
                 }
             }
+            rs.close();
             if (lastTenantId != null) {
                 conn.close();
             }
@@ -860,6 +860,7 @@ public abstract class BaseTest {
             logger.info("DROP SEQUENCE STATEMENT: DROP SEQUENCE " + SchemaUtil.getEscapedTableName(rs.getString(2), rs.getString(3)));
             conn.createStatement().execute("DROP SEQUENCE " + SchemaUtil.getEscapedTableName(rs.getString(2), rs.getString(3)));
         }
+        rs.close();
     }
     
     protected static void initSumDoubleValues(byte[][] splits, String url) throws Exception {


[06/31] phoenix git commit: Changing version to 4.5.0-HBase-1.1-SNAPSHOT

Posted by nd...@apache.org.
Changing version to 4.5.0-HBase-1.1-SNAPSHOT


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

Branch: refs/heads/4.x-HBase-1.1
Commit: 3cdc3230c570ee8c22bb6c1bab975699fd02e94c
Parents: 56e1c0a
Author: Rajeshbabu Chintaguntla <ra...@apache.org>
Authored: Mon May 25 17:46:18 2015 +0530
Committer: Rajeshbabu Chintaguntla <ra...@apache.org>
Committed: Mon May 25 17:46:18 2015 +0530

----------------------------------------------------------------------
 phoenix-assembly/pom.xml      | 2 +-
 phoenix-core/pom.xml          | 2 +-
 phoenix-flume/pom.xml         | 2 +-
 phoenix-pherf/pom.xml         | 2 +-
 phoenix-pig/pom.xml           | 2 +-
 phoenix-server-client/pom.xml | 2 +-
 phoenix-server/pom.xml        | 2 +-
 phoenix-spark/pom.xml         | 2 +-
 pom.xml                       | 2 +-
 9 files changed, 9 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/3cdc3230/phoenix-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-assembly/pom.xml b/phoenix-assembly/pom.xml
index 8d9a965..04d9335 100644
--- a/phoenix-assembly/pom.xml
+++ b/phoenix-assembly/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.4.0-SNAPSHOT</version>
+    <version>4.5.0-HBase-1.1-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-assembly</artifactId>
   <name>Phoenix Assembly</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3cdc3230/phoenix-core/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-core/pom.xml b/phoenix-core/pom.xml
index 22e6b60..951e969 100644
--- a/phoenix-core/pom.xml
+++ b/phoenix-core/pom.xml
@@ -4,7 +4,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.4.0-SNAPSHOT</version>
+    <version>4.5.0-HBase-1.1-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-core</artifactId>
   <name>Phoenix Core</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3cdc3230/phoenix-flume/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-flume/pom.xml b/phoenix-flume/pom.xml
index b2b9a47..ea87ab0 100644
--- a/phoenix-flume/pom.xml
+++ b/phoenix-flume/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.4.0-SNAPSHOT</version>
+    <version>4.5.0-HBase-1.1-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-flume</artifactId>
   <name>Phoenix - Flume</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3cdc3230/phoenix-pherf/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-pherf/pom.xml b/phoenix-pherf/pom.xml
index 0901f71..e751d73 100644
--- a/phoenix-pherf/pom.xml
+++ b/phoenix-pherf/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <groupId>org.apache.phoenix</groupId>
         <artifactId>phoenix</artifactId>
-        <version>4.4.0-SNAPSHOT</version>
+        <version>4.5.0-HBase-1.1-SNAPSHOT</version>
     </parent>
 
     <artifactId>phoenix-pherf</artifactId>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3cdc3230/phoenix-pig/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-pig/pom.xml b/phoenix-pig/pom.xml
index 015a660..957c06f 100644
--- a/phoenix-pig/pom.xml
+++ b/phoenix-pig/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.4.0-SNAPSHOT</version>
+    <version>4.5.0-HBase-1.1-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-pig</artifactId>
   <name>Phoenix - Pig</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3cdc3230/phoenix-server-client/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-server-client/pom.xml b/phoenix-server-client/pom.xml
index 4d6fd45..748e57c 100644
--- a/phoenix-server-client/pom.xml
+++ b/phoenix-server-client/pom.xml
@@ -4,7 +4,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.4.0-SNAPSHOT</version>
+    <version>4.5.0-HBase-1.1-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-server-client</artifactId>
   <name>Phoenix Query Server Client</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3cdc3230/phoenix-server/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-server/pom.xml b/phoenix-server/pom.xml
index 9f6289f..ab9a472 100644
--- a/phoenix-server/pom.xml
+++ b/phoenix-server/pom.xml
@@ -4,7 +4,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.4.0-SNAPSHOT</version>
+    <version>4.5.0-HBase-1.1-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-server</artifactId>
   <name>Phoenix Query Server</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3cdc3230/phoenix-spark/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-spark/pom.xml b/phoenix-spark/pom.xml
index a232cf4..e42a7ec 100644
--- a/phoenix-spark/pom.xml
+++ b/phoenix-spark/pom.xml
@@ -28,7 +28,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.4.0-SNAPSHOT</version>
+    <version>4.5.0-HBase-1.1-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-spark</artifactId>
   <name>Phoenix - Spark</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3cdc3230/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 4361e54..f6708cc 100644
--- a/pom.xml
+++ b/pom.xml
@@ -3,7 +3,7 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>org.apache.phoenix</groupId>
   <artifactId>phoenix</artifactId>
-  <version>4.4.0-SNAPSHOT</version>
+  <version>4.5.0-HBase-1.1-SNAPSHOT</version>
   <packaging>pom</packaging>
   <name>Apache Phoenix</name>
   <description>A SQL layer over HBase</description>


[11/31] phoenix git commit: PHOENIX-1964 - porting from master

Posted by nd...@apache.org.
PHOENIX-1964 - porting from master


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

Branch: refs/heads/4.x-HBase-1.1
Commit: c95e28df94241f47d5cfe9a1515b21960c93adf2
Parents: 0e0b4dd
Author: cmarcel <cm...@salesforce.com>
Authored: Wed May 27 13:58:45 2015 -0700
Committer: cmarcel <cm...@salesforce.com>
Committed: Wed May 27 13:58:45 2015 -0700

----------------------------------------------------------------------
 phoenix-pherf/config/pherf.properties           |  3 ++
 .../org/apache/phoenix/pherf/DataIngestIT.java  |  3 +-
 .../apache/phoenix/pherf/ResultBaseTestIT.java  | 45 ++++++++++++++++++
 .../java/org/apache/phoenix/pherf/Pherf.java    |  7 +--
 .../apache/phoenix/pherf/PherfConstants.java    | 50 +++++++++++++++++++-
 .../phoenix/pherf/loaddata/DataLoader.java      |  2 +-
 .../apache/phoenix/pherf/result/ResultUtil.java |  4 +-
 .../pherf/result/impl/CSVResultHandler.java     |  5 +-
 .../pherf/result/impl/ImageResultHandler.java   |  5 +-
 .../pherf/result/impl/XMLResultHandler.java     |  6 ++-
 .../apache/phoenix/pherf/util/ResourceList.java | 26 ----------
 .../pherf/workload/WorkloadExecutor.java        |  2 +-
 .../phoenix/pherf/ConfigurationParserTest.java  |  2 +-
 .../org/apache/phoenix/pherf/ResourceTest.java  |  8 ++--
 .../apache/phoenix/pherf/ResultBaseTest.java    | 44 +++++++++++++++++
 .../org/apache/phoenix/pherf/ResultTest.java    |  5 +-
 16 files changed, 168 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/c95e28df/phoenix-pherf/config/pherf.properties
----------------------------------------------------------------------
diff --git a/phoenix-pherf/config/pherf.properties b/phoenix-pherf/config/pherf.properties
index 354707a..1142f9b5 100644
--- a/phoenix-pherf/config/pherf.properties
+++ b/phoenix-pherf/config/pherf.properties
@@ -29,3 +29,6 @@ pherf.default.dataloader.threadpool=0
 # When upserting, this is the max # of rows that will be inserted in a single commit
 pherf.default.dataloader.batchsize=1000
 
+# Directory where results from a scenario run will be written
+pherf.default.results.dir=RESULTS
+

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c95e28df/phoenix-pherf/src/it/java/org/apache/phoenix/pherf/DataIngestIT.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/it/java/org/apache/phoenix/pherf/DataIngestIT.java b/phoenix-pherf/src/it/java/org/apache/phoenix/pherf/DataIngestIT.java
index b29656d..2b56f43 100644
--- a/phoenix-pherf/src/it/java/org/apache/phoenix/pherf/DataIngestIT.java
+++ b/phoenix-pherf/src/it/java/org/apache/phoenix/pherf/DataIngestIT.java
@@ -18,7 +18,6 @@
 
 package org.apache.phoenix.pherf;
 
-import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
 import org.apache.phoenix.pherf.configuration.Column;
 import org.apache.phoenix.pherf.configuration.DataTypeMapping;
 import org.apache.phoenix.pherf.configuration.Scenario;
@@ -39,7 +38,7 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-public class DataIngestIT extends BaseHBaseManagedTimeIT {
+public class DataIngestIT extends ResultBaseTestIT {
     protected static PhoenixUtil util = new PhoenixUtil(true);
     static final String matcherScenario = ".*scenario/.*test.*xml";
     static final String matcherSchema = ".*datamodel/.*test.*sql";

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c95e28df/phoenix-pherf/src/it/java/org/apache/phoenix/pherf/ResultBaseTestIT.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/it/java/org/apache/phoenix/pherf/ResultBaseTestIT.java b/phoenix-pherf/src/it/java/org/apache/phoenix/pherf/ResultBaseTestIT.java
new file mode 100644
index 0000000..6e103b8
--- /dev/null
+++ b/phoenix-pherf/src/it/java/org/apache/phoenix/pherf/ResultBaseTestIT.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *   or more contributor license agreements.  See the NOTICE file
+ *   distributed with this work for additional information
+ *   regarding copyright ownership.  The ASF licenses this file
+ *   to you under the Apache License, Version 2.0 (the
+ *   "License"); you may not use this file except in compliance
+ *   with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ */
+
+package org.apache.phoenix.pherf;
+
+import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.pherf.result.ResultUtil;
+import org.junit.BeforeClass;
+
+import java.util.Properties;
+
+public class ResultBaseTestIT extends BaseHBaseManagedTimeIT {
+    private static boolean isSetUpDone = false;
+
+    @BeforeClass
+    public static void setUp() throws Exception {
+        if (isSetUpDone) {
+            return;
+        }
+
+        ResultUtil util = new ResultUtil();
+        PherfConstants constants = PherfConstants.create();
+        Properties properties = constants.getProperties(PherfConstants.PHERF_PROPERTIES);
+        String dir = properties.getProperty("pherf.default.results.dir");
+        String targetDir = "target/" + dir;
+        properties.setProperty("pherf.default.results.dir", targetDir);
+        util.ensureBaseDirExists(targetDir);
+        isSetUpDone = true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c95e28df/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/Pherf.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/Pherf.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/Pherf.java
index 0ccc49a..073c661 100644
--- a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/Pherf.java
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/Pherf.java
@@ -84,7 +84,7 @@ public class Pherf {
             System.exit(1);
         }
 
-        properties = getProperties();
+        properties = PherfConstants.create().getProperties(PherfConstants.PHERF_PROPERTIES);
         dropPherfTablesRegEx = command.getOptionValue("drop", null);
         monitor = command.hasOption("m");
         String monitorFrequency = (command.hasOption("m") && command.hasOption("monitorFrequency"))
@@ -192,9 +192,4 @@ public class Pherf {
             }
         }
     }
-
-    private static Properties getProperties() throws Exception {
-        ResourceList list = new ResourceList();
-        return list.getProperties();
-    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c95e28df/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/PherfConstants.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/PherfConstants.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/PherfConstants.java
index 22d18f6..493f5a8 100644
--- a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/PherfConstants.java
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/PherfConstants.java
@@ -18,7 +18,13 @@
 
 package org.apache.phoenix.pherf;
 
+import java.io.InputStream;
+import java.util.Properties;
+
 public class PherfConstants {
+    private static PherfConstants instance = null;
+    private Properties properties = null;
+
     public static final int DEFAULT_THREAD_POOL_SIZE = 10;
     public static final int DEFAULT_BATCH_SIZE = 1000;
     public static final String DEFAULT_DATE_PATTERN = "yyyy-MM-dd HH:mm:ss.SSS";
@@ -29,7 +35,7 @@ public class PherfConstants {
             ".*" + PherfConstants.RESOURCE_SCENARIO.substring(1) + ".*";
     public static final String SCHEMA_ROOT_PATTERN = ".*";
     public static final String PHERF_PROPERTIES = "pherf.properties";
-    public static final String RESULT_DIR = "RESULTS";
+//    public static final String RESULT_DIR = "RESULTS";
     public static final String EXPORT_DIR = "CSV_EXPORT";
     public static final String RESULT_PREFIX = "RESULT_";
     public static final String PATH_SEPARATOR = "/";
@@ -60,4 +66,46 @@ public class PherfConstants {
         PERFORMANCE,
         FUNCTIONAL
     }
+
+    private PherfConstants() {
+    }
+
+    public static PherfConstants create() {
+        if (instance == null) {
+            instance = new PherfConstants();
+        }
+        return instance;
+    }
+
+    public Properties getProperties(final String fileName) throws Exception {
+        if (properties != null) {
+            return properties;
+        }
+
+        properties = new Properties();
+        InputStream is = null;
+        try {
+            is = getClass().getClassLoader().getResourceAsStream(fileName);
+            properties.load(is);
+        } finally {
+            if (is != null) {
+                is.close();
+            }
+        }
+        return properties;
+    }
+
+    public String getProperty(String property) {
+        return getProperty(PherfConstants.PHERF_PROPERTIES, property);
+    }
+
+    public String getProperty(final String fileName, String property) {
+        String value = null;
+        try {
+            value = getProperties(fileName).getProperty(property);
+        } catch (Exception e) {
+            e.printStackTrace();
+        }
+        return value;
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c95e28df/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/loaddata/DataLoader.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/loaddata/DataLoader.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/loaddata/DataLoader.java
index abec1b4..c521822 100644
--- a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/loaddata/DataLoader.java
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/loaddata/DataLoader.java
@@ -67,7 +67,7 @@ public class DataLoader {
     }
 
     public DataLoader(PhoenixUtil phoenixUtil, XMLConfigParser parser) throws Exception{
-        this(phoenixUtil, new ResourceList().getProperties(), parser);
+        this(phoenixUtil, PherfConstants.create().getProperties(PherfConstants.PHERF_PROPERTIES), parser);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c95e28df/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/ResultUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/ResultUtil.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/ResultUtil.java
index 360cb49..fd960d1 100644
--- a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/ResultUtil.java
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/ResultUtil.java
@@ -115,7 +115,9 @@ public class ResultUtil {
     }
 
     public void ensureBaseResultDirExists() {
-        ensureBaseDirExists(PherfConstants.RESULT_DIR);
+        PherfConstants constants = PherfConstants.create();
+        String resultDir = constants.getProperty("pherf.default.results.dir");
+        ensureBaseDirExists(resultDir);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c95e28df/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/impl/CSVResultHandler.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/impl/CSVResultHandler.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/impl/CSVResultHandler.java
index ca470de..e7fbb48 100644
--- a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/impl/CSVResultHandler.java
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/impl/CSVResultHandler.java
@@ -53,8 +53,11 @@ public class CSVResultHandler implements ResultHandler {
 
     public CSVResultHandler(String resultFileName, ResultFileDetails resultFileDetails, boolean generateFullFileName) {
         this.util = new ResultUtil();
+        PherfConstants constants = PherfConstants.create();
+        String resultDir = constants.getProperty("pherf.default.results.dir");
+
         this.resultFileName = generateFullFileName ?
-                PherfConstants.RESULT_DIR + PherfConstants.PATH_SEPARATOR
+                resultDir + PherfConstants.PATH_SEPARATOR
                         + PherfConstants.RESULT_PREFIX
                         + resultFileName + util.getSuffix()
                         + resultFileDetails.getExtension().toString()

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c95e28df/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/impl/ImageResultHandler.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/impl/ImageResultHandler.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/impl/ImageResultHandler.java
index 6e66cf6..ad3c8fb 100644
--- a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/impl/ImageResultHandler.java
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/impl/ImageResultHandler.java
@@ -44,8 +44,11 @@ public class ImageResultHandler implements ResultHandler {
 
     public ImageResultHandler(String resultFileName, ResultFileDetails resultFileDetails, boolean generateFullFileName) {
         ResultUtil util = new ResultUtil();
+        PherfConstants constants = PherfConstants.create();
+        String resultDir = constants.getProperty("pherf.default.results.dir");
+
         this.resultFileName = generateFullFileName ?
-                PherfConstants.RESULT_DIR + PherfConstants.PATH_SEPARATOR
+                resultDir + PherfConstants.PATH_SEPARATOR
                         + PherfConstants.RESULT_PREFIX
                         + resultFileName + util.getSuffix()
                         + resultFileDetails.getExtension().toString()

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c95e28df/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/impl/XMLResultHandler.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/impl/XMLResultHandler.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/impl/XMLResultHandler.java
index a1d0930..8a913ed 100644
--- a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/impl/XMLResultHandler.java
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/result/impl/XMLResultHandler.java
@@ -30,6 +30,7 @@ import java.io.FileOutputStream;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Properties;
 
 public class XMLResultHandler implements ResultHandler {
     private final String resultFileName;
@@ -41,8 +42,11 @@ public class XMLResultHandler implements ResultHandler {
 
     public XMLResultHandler(String resultFileName, ResultFileDetails resultFileDetails, boolean generateFullFileName) {
         ResultUtil util = new ResultUtil();
+        PherfConstants constants = PherfConstants.create();
+        String resultDir = constants.getProperty("pherf.default.results.dir");
+
         this.resultFileName = generateFullFileName ?
-                PherfConstants.RESULT_DIR + PherfConstants.PATH_SEPARATOR
+                resultDir + PherfConstants.PATH_SEPARATOR
                         + PherfConstants.RESULT_PREFIX
                         + resultFileName + util.getSuffix()
                         + resultFileDetails.getExtension().toString()

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c95e28df/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/util/ResourceList.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/util/ResourceList.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/util/ResourceList.java
index ac1e74d..5359c35 100644
--- a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/util/ResourceList.java
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/util/ResourceList.java
@@ -18,14 +18,12 @@
 
 package org.apache.phoenix.pherf.util;
 
-import org.apache.phoenix.pherf.PherfConstants;
 import org.apache.phoenix.pherf.exception.PherfException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.File;
 import java.io.IOException;
-import java.io.InputStream;
 import java.net.URI;
 import java.net.URL;
 import java.nio.file.Path;
@@ -43,17 +41,11 @@ public class ResourceList {
     private static final Logger logger = LoggerFactory.getLogger(ResourceList.class);
     private final String rootResourceDir;
 
-    public ResourceList() {
-        this("/");
-    }
-
     public ResourceList(String rootResourceDir) {
         this.rootResourceDir = rootResourceDir;
     }
 
     public Collection<Path> getResourceList(final String pattern) throws Exception {
-        Properties properties = getProperties();
-
         // Include files from config directory
         Collection<Path> paths = getResourcesPaths(Pattern.compile(pattern));
 
@@ -112,24 +104,6 @@ public class ResourceList {
         return paths;
     }
 
-    public Properties getProperties() throws Exception {
-        return getProperties(PherfConstants.PHERF_PROPERTIES);
-    }
-
-    public Properties getProperties(final String fileName) throws Exception {
-        Properties pherfProps = new Properties();
-        InputStream is = null;
-        try {
-            is = getClass().getClassLoader().getResourceAsStream(fileName);
-            pherfProps.load(is);
-        } finally {
-            if (is != null) {
-                is.close();
-            }
-        }
-        return pherfProps;
-    }
-
     private Collection<String> getResources(
             final String element,
             final Pattern pattern) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c95e28df/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/WorkloadExecutor.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/WorkloadExecutor.java b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/WorkloadExecutor.java
index 0509bdc..cf2f038 100644
--- a/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/WorkloadExecutor.java
+++ b/phoenix-pherf/src/main/java/org/apache/phoenix/pherf/workload/WorkloadExecutor.java
@@ -45,7 +45,7 @@ public class WorkloadExecutor {
 
 
     public WorkloadExecutor() throws Exception {
-        this(new ResourceList().getProperties());
+        this(PherfConstants.create().getProperties(PherfConstants.PHERF_PROPERTIES));
     }
 
     public WorkloadExecutor(Properties properties) throws Exception{

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c95e28df/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/ConfigurationParserTest.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/ConfigurationParserTest.java b/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/ConfigurationParserTest.java
index fcc353e..f362842 100644
--- a/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/ConfigurationParserTest.java
+++ b/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/ConfigurationParserTest.java
@@ -38,7 +38,7 @@ import javax.xml.bind.Marshaller;
 
 import static org.junit.Assert.*;
 
-public class ConfigurationParserTest {
+public class ConfigurationParserTest extends ResultBaseTest{
     private static final Logger logger = LoggerFactory.getLogger(ConfigurationParserTest.class);
 
     @Test

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c95e28df/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/ResourceTest.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/ResourceTest.java b/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/ResourceTest.java
index 042f9c3..cd567cb 100644
--- a/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/ResourceTest.java
+++ b/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/ResourceTest.java
@@ -48,11 +48,9 @@ public class ResourceTest {
 
     @Test
     public void testResourceListPropertyDirectory() throws Exception {
-
-        ResourceList list = new ResourceList();
-        Properties properties = list.getProperties();
-        assertTrue("Property file list was empty", properties.size() > 0);
-        assertNotNull(properties.getProperty("pherf.default.dataloader.threadpool"));
+        PherfConstants constants = PherfConstants.create();
+        assertNotNull(constants.getProperty("pherf.default.dataloader.threadpool"));
+        assertNotNull(constants.getProperty("pherf.default.results.dir"));
     }
 
     private Collection<Path> assertResources(String pattern, String rootDir, String assertStr) throws Exception {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c95e28df/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/ResultBaseTest.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/ResultBaseTest.java b/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/ResultBaseTest.java
new file mode 100644
index 0000000..5c455fc
--- /dev/null
+++ b/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/ResultBaseTest.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *   or more contributor license agreements.  See the NOTICE file
+ *   distributed with this work for additional information
+ *   regarding copyright ownership.  The ASF licenses this file
+ *   to you under the Apache License, Version 2.0 (the
+ *   "License"); you may not use this file except in compliance
+ *   with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *   Unless required by applicable law or agreed to in writing, software
+ *   distributed under the License is distributed on an "AS IS" BASIS,
+ *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *   See the License for the specific language governing permissions and
+ *   limitations under the License.
+ */
+
+package org.apache.phoenix.pherf;
+
+import org.apache.phoenix.pherf.result.ResultUtil;
+import org.junit.BeforeClass;
+
+import java.util.Properties;
+
+public class ResultBaseTest {
+    private static boolean isSetUpDone = false;
+
+    @BeforeClass
+    public static void setUp() throws Exception {
+        if (isSetUpDone) {
+            return;
+        }
+
+        ResultUtil util = new ResultUtil();
+        PherfConstants constants = PherfConstants.create();
+        Properties properties = constants.getProperties(PherfConstants.PHERF_PROPERTIES);
+        String dir = properties.getProperty("pherf.default.results.dir");
+        String targetDir = "target/" + dir;
+        properties.setProperty("pherf.default.results.dir", targetDir);
+        util.ensureBaseDirExists(targetDir);
+        isSetUpDone = true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c95e28df/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/ResultTest.java
----------------------------------------------------------------------
diff --git a/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/ResultTest.java b/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/ResultTest.java
index 0f4dfd1..4ab36fb 100644
--- a/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/ResultTest.java
+++ b/phoenix-pherf/src/test/java/org/apache/phoenix/pherf/ResultTest.java
@@ -33,11 +33,12 @@ import org.apache.phoenix.pherf.result.file.ResultFileDetails;
 import org.apache.phoenix.pherf.result.impl.CSVResultHandler;
 import org.apache.phoenix.pherf.result.impl.XMLResultHandler;
 import org.apache.phoenix.pherf.result.*;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.phoenix.pherf.configuration.Query;
 
-public class ResultTest {
+public class ResultTest extends ResultBaseTest {
 
     @Test
     public void testMonitorWriter() throws Exception {
@@ -96,7 +97,7 @@ public class ResultTest {
         records = monitor.readResults();
 
         assertNotNull("Could not retrieve records", records);
-        assertEquals("Failed to get correct amount of CSV records.", records.size(), monitor.getRowCount());
+        assertTrue("Failed to get correct CSV records.", records.size() > 0);
         assertFalse("Monitor was not stopped correctly.", monitor.isRunning());
     }
 


[16/31] phoenix git commit: PHOENIX-2016 Some Phoenix tests failed with NPE(Alicia Ying Shu)

Posted by nd...@apache.org.
PHOENIX-2016 Some Phoenix tests failed with NPE(Alicia Ying Shu)


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

Branch: refs/heads/4.x-HBase-1.1
Commit: dc46b144aa9eaf315c3969669dab7f0a50d94281
Parents: eb9452d
Author: Rajeshbabu Chintaguntla <ra...@apache.org>
Authored: Mon Jun 1 21:34:16 2015 +0530
Committer: Rajeshbabu Chintaguntla <ra...@apache.org>
Committed: Mon Jun 1 21:34:16 2015 +0530

----------------------------------------------------------------------
 phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/dc46b144/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
index b0574c3..fa78656 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
@@ -1627,6 +1627,7 @@ public abstract class BaseTest {
      * Disable and drop all the tables except SYSTEM.CATALOG and SYSTEM.SEQUENCE
      */
     private static void disableAndDropNonSystemTables() throws Exception {
+        if (driver == null) return;
         HBaseAdmin admin = driver.getConnectionQueryServices(null, null).getAdmin();
         try {
             HTableDescriptor[] tables = admin.listTables();