You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ja...@apache.org on 2017/11/09 20:58:09 UTC

[01/20] phoenix git commit: PHOENIX-4289 UPDATE STATISTICS command does not collect stats for local indexes

Repository: phoenix
Updated Branches:
  refs/heads/5.x-HBase-2.0 754201cfb -> 0454e4211


PHOENIX-4289 UPDATE STATISTICS command does not collect stats for local indexes


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

Branch: refs/heads/5.x-HBase-2.0
Commit: f5131945c75d85108887c49ed5c294a01a28e095
Parents: 754201c
Author: Samarth Jain <sa...@apache.org>
Authored: Sun Oct 29 22:59:03 2017 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Thu Nov 9 12:41:21 2017 -0800

----------------------------------------------------------------------
 .../end2end/ExplainPlanWithStatsEnabledIT.java  | 13 ++++
 .../phoenix/end2end/index/BaseLocalIndexIT.java |  3 +
 .../phoenix/end2end/index/LocalIndexIT.java     | 46 ++++++++++-
 .../phoenix/iterate/BaseResultIterators.java    |  4 +-
 .../apache/phoenix/schema/MetaDataClient.java   | 80 +++++++++++++-------
 5 files changed, 115 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/f5131945/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
index cd4555c..62538af 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
@@ -32,6 +32,7 @@ import java.util.List;
 
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.query.BaseTest;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.TableNotFoundException;
 import org.apache.phoenix.util.EnvironmentEdge;
@@ -306,6 +307,18 @@ public class ExplainPlanWithStatsEnabledIT extends ParallelStatsEnabledIT {
         final Long estimatedRows;
         final Long estimateInfoTs;
 
+        public Long getEstimatedBytes() {
+            return estimatedBytes;
+        }
+
+        public Long getEstimatedRows() {
+            return estimatedRows;
+        }
+
+        public Long getEstimateInfoTs() {
+            return estimateInfoTs;
+        }
+
         Estimate(Long rows, Long bytes, Long ts) {
             this.estimatedBytes = bytes;
             this.estimatedRows = rows;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f5131945/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseLocalIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseLocalIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseLocalIndexIT.java
index 30baec4..1659d73 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseLocalIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseLocalIndexIT.java
@@ -59,6 +59,9 @@ public abstract class BaseLocalIndexIT extends BaseUniqueNamesOwnClusterIT {
         serverProps.put(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, "true");
         Map<String, String> clientProps = Maps.newHashMapWithExpectedSize(1);
         clientProps.put(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, "true");
+        // setting update frequency to a large value to test out that we are
+        // generating stats for local indexes
+        clientProps.put(QueryServices.MIN_STATS_UPDATE_FREQ_MS_ATTRIB, "120000");
         setUpTestDriver(new ReadOnlyProps(serverProps.entrySet().iterator()), new ReadOnlyProps(clientProps.entrySet().iterator()));
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f5131945/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java
index 238b88e..f97ba22 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/LocalIndexIT.java
@@ -17,6 +17,7 @@
  */
 package org.apache.phoenix.end2end.index;
 
+import static org.apache.phoenix.end2end.ExplainPlanWithStatsEnabledIT.getByteRowEstimates;
 import static org.apache.phoenix.util.MetaDataUtil.getViewIndexSequenceName;
 import static org.apache.phoenix.util.MetaDataUtil.getViewIndexSequenceSchemaName;
 import static org.junit.Assert.assertArrayEquals;
@@ -56,8 +57,10 @@ import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.compile.QueryPlan;
 import org.apache.phoenix.coprocessor.BaseScannerRegionObserver;
+import org.apache.phoenix.end2end.ExplainPlanWithStatsEnabledIT.Estimate;
 import org.apache.phoenix.hbase.index.IndexRegionSplitPolicy;
 import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixResultSet;
 import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.PNameFactory;
@@ -68,9 +71,10 @@ import org.apache.phoenix.schema.TableNotFoundException;
 import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.TestUtil;
-import org.junit.Ignore;
 import org.junit.Test;
 
+import com.google.common.collect.Lists;
+
 public class LocalIndexIT extends BaseLocalIndexIT {
     public LocalIndexIT(boolean isNamespaceMapped) {
         super(isNamespaceMapped);
@@ -722,4 +726,44 @@ public class LocalIndexIT extends BaseLocalIndexIT {
         }
     }
 
+    @Test // See https://issues.apache.org/jira/browse/PHOENIX-4289
+    public void testEstimatesWithLocalIndexes() throws Exception {
+        String tableName = generateUniqueName();
+        String indexName = "IDX_" + generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            int guidePostWidth = 20;
+            conn.createStatement()
+                    .execute("CREATE TABLE " + tableName
+                            + " (k INTEGER PRIMARY KEY, a bigint, b bigint)"
+                            + " GUIDE_POSTS_WIDTH=" + guidePostWidth);
+            conn.createStatement().execute("upsert into " + tableName + " values (100,1,3)");
+            conn.createStatement().execute("upsert into " + tableName + " values (101,2,4)");
+            conn.createStatement().execute("upsert into " + tableName + " values (102,2,4)");
+            conn.createStatement().execute("upsert into " + tableName + " values (103,2,4)");
+            conn.createStatement().execute("upsert into " + tableName + " values (104,2,4)");
+            conn.createStatement().execute("upsert into " + tableName + " values (105,2,4)");
+            conn.createStatement().execute("upsert into " + tableName + " values (106,2,4)");
+            conn.createStatement().execute("upsert into " + tableName + " values (107,2,4)");
+            conn.createStatement().execute("upsert into " + tableName + " values (108,2,4)");
+            conn.createStatement().execute("upsert into " + tableName + " values (109,2,4)");
+            conn.commit();
+            conn.createStatement().execute(
+                "CREATE LOCAL INDEX " + indexName + " ON " + tableName + " (a) INCLUDE (b) ");
+            String ddl = "ALTER TABLE " + tableName + " SET USE_STATS_FOR_PARALLELIZATION = false";
+            conn.createStatement().execute(ddl);
+            conn.createStatement().execute("UPDATE STATISTICS " + tableName + "");
+        }
+        List<Object> binds = Lists.newArrayList();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String sql =
+                    "SELECT COUNT(*) " + " FROM " + tableName;
+            ResultSet rs = conn.createStatement().executeQuery(sql);
+            assertTrue("Index " + indexName + " should have been used",
+                rs.unwrap(PhoenixResultSet.class).getStatement().getQueryPlan().getTableRef()
+                        .getTable().getName().getString().equals(indexName));
+            Estimate info = getByteRowEstimates(conn, sql, binds);
+            assertEquals((Long) 10l, info.getEstimatedRows());
+            assertTrue(info.getEstimateInfoTs() > 0);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f5131945/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
index 46fd55c..9bf9573 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
@@ -491,9 +491,7 @@ public abstract class BaseResultIterators extends ExplainTable implements Result
         scanId = new UUID(ThreadLocalRandom.current().nextLong(), ThreadLocalRandom.current().nextLong()).toString();
         
         initializeScan(plan, perScanLimit, offset, scan);
-        this.useStatsForParallelization =
-                context.getConnection().getQueryServices().getConfiguration().getBoolean(
-                    USE_STATS_FOR_PARALLELIZATION, DEFAULT_USE_STATS_FOR_PARALLELIZATION);
+        this.useStatsForParallelization = table.useStatsForParallelization();
         this.scans = getParallelScans();
         List<KeyRange> splitRanges = Lists.newArrayListWithExpectedSize(scans.size() * ESTIMATED_GUIDEPOSTS_PER_REGION);
         for (List<Scan> scanList : scans) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f5131945/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 0ce4246..701633b 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
@@ -236,7 +236,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Strings;
-import com.google.common.collect.Iterators;
 import com.google.common.collect.ListMultimap;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
@@ -1088,7 +1087,7 @@ public class MetaDataClient {
         PTable table = resolver.getTables().get(0).getTable();
         long rowCount = 0;
         if (updateStatisticsStmt.updateColumns()) {
-            rowCount += updateStatisticsInternal(table.getPhysicalName(), table, updateStatisticsStmt.getProps());
+            rowCount += updateStatisticsInternal(table.getPhysicalName(), table, updateStatisticsStmt.getProps(), true);
         }
         if (updateStatisticsStmt.updateIndex()) {
             // TODO: If our table is a VIEW with multiple indexes or a TABLE with local indexes,
@@ -1096,25 +1095,50 @@ public class MetaDataClient {
             // across all indexes in that case so that we don't re-calculate the same stats
             // multiple times.
             for (PTable index : table.getIndexes()) {
-                rowCount += updateStatisticsInternal(index.getPhysicalName(), index, updateStatisticsStmt.getProps());
+                // If the table is a view, then we will end up calling update stats
+                // here for all the view indexes on it. We take care of local indexes later.
+                if (index.getIndexType() != IndexType.LOCAL) {
+                    rowCount += updateStatisticsInternal(table.getPhysicalName(), index, updateStatisticsStmt.getProps(), true);
+                }
+            }
+            /*
+             * Update stats for local indexes. This takes care of local indexes on the the table
+             * as well as local indexes on any views on it.
+             */
+            PName physicalName = table.getPhysicalName();
+            List<byte[]> localCFs = MetaDataUtil.getLocalIndexColumnFamilies(connection, physicalName.getBytes());
+            if (!localCFs.isEmpty()) {
+                /*
+                 * We need to pass checkLastStatsUpdateTime as false here. Local indexes are on the
+                 * same table as the physical table. So when the user has requested to update stats
+                 * for both table and indexes on it, we need to make sure that we don't re-check
+                 * LAST_UPDATE_STATS time. If we don't do that then we will end up *not* collecting
+                 * stats for local indexes which would be bad.
+                 *
+                 * Note, that this also means we don't have a way of controlling how often update
+                 * stats can run for local indexes. Consider the case when the user calls UPDATE STATS TABLE
+                 * followed by UPDATE STATS TABLE INDEX. When the second statement is being executed,
+                 * this causes us to skip the check and execute stats collection possibly a bit too frequently.
+                 */
+                rowCount += updateStatisticsInternal(physicalName, table, updateStatisticsStmt.getProps(), localCFs, false);
             }
             // If analyzing the indexes of a multi-tenant table or a table with view indexes
             // then analyze all of those indexes too.
             if (table.getType() != PTableType.VIEW) {
                 if (table.isMultiTenant() || MetaDataUtil.hasViewIndexTable(connection, table.getPhysicalName())) {
-                    final PName physicalName = PNameFactory.newName(MetaDataUtil.getViewIndexPhysicalName(table.getPhysicalName().getBytes()));
+                    final PName viewIndexPhysicalTableName = PNameFactory.newName(MetaDataUtil.getViewIndexPhysicalName(table.getPhysicalName().getBytes()));
                     PTable indexLogicalTable = new DelegateTable(table) {
                         @Override
                         public PName getPhysicalName() {
-                            return physicalName;
+                            return viewIndexPhysicalTableName;
                         }
                     };
-                    rowCount += updateStatisticsInternal(physicalName, indexLogicalTable, updateStatisticsStmt.getProps());
-                }
-                PName physicalName = table.getPhysicalName();
-                List<byte[]> localCFs = MetaDataUtil.getLocalIndexColumnFamilies(connection, physicalName.getBytes());
-                if (!localCFs.isEmpty()) {
-                    rowCount += updateStatisticsInternal(physicalName, table, updateStatisticsStmt.getProps(), localCFs);
+                    /*
+                     * Note for future maintainers: local indexes whether on a table or on a view,
+                     * reside on the same physical table as the base table and not the view index
+                     * table. So below call is collecting stats only for non-local view indexes.
+                     */
+                    rowCount += updateStatisticsInternal(viewIndexPhysicalTableName, indexLogicalTable, updateStatisticsStmt.getProps(), true);
                 }
             }
         }
@@ -1127,27 +1151,29 @@ public class MetaDataClient {
         };
     }
 
-    private long updateStatisticsInternal(PName physicalName, PTable logicalTable, Map<String, Object> statsProps) throws SQLException {
-        return updateStatisticsInternal(physicalName, logicalTable, statsProps, null);
+    private long updateStatisticsInternal(PName physicalName, PTable logicalTable, Map<String, Object> statsProps, boolean checkLastStatsUpdateTime) throws SQLException {
+        return updateStatisticsInternal(physicalName, logicalTable, statsProps, null, checkLastStatsUpdateTime);
     }
     
-    private long updateStatisticsInternal(PName physicalName, PTable logicalTable, Map<String, Object> statsProps, List<byte[]> cfs) throws SQLException {
+    private long updateStatisticsInternal(PName physicalName, PTable logicalTable, Map<String, Object> statsProps, List<byte[]> cfs, boolean checkLastStatsUpdateTime) throws SQLException {
         ReadOnlyProps props = connection.getQueryServices().getProps();
         final long msMinBetweenUpdates = props
                 .getLong(QueryServices.MIN_STATS_UPDATE_FREQ_MS_ATTRIB,
                         props.getLong(QueryServices.STATS_UPDATE_FREQ_MS_ATTRIB,
                                 QueryServicesOptions.DEFAULT_STATS_UPDATE_FREQ_MS) / 2);
-        byte[] tenantIdBytes = ByteUtil.EMPTY_BYTE_ARRAY;
         Long scn = connection.getSCN();
         // Always invalidate the cache
         long clientTimeStamp = connection.getSCN() == null ? HConstants.LATEST_TIMESTAMP : scn;
-        String query = "SELECT CURRENT_DATE()," + LAST_STATS_UPDATE_TIME + " FROM " + PhoenixDatabaseMetaData.SYSTEM_STATS_NAME
-                + " WHERE " + PHYSICAL_NAME + "='" + physicalName.getString() + "' AND " + COLUMN_FAMILY
-                + " IS NULL AND " + LAST_STATS_UPDATE_TIME + " IS NOT NULL";
-        ResultSet rs = connection.createStatement().executeQuery(query);
         long msSinceLastUpdate = Long.MAX_VALUE;
-        if (rs.next()) {
-            msSinceLastUpdate = rs.getLong(1) - rs.getLong(2);
+        if (checkLastStatsUpdateTime) {
+            String query = "SELECT CURRENT_DATE()," + LAST_STATS_UPDATE_TIME + " FROM " + PhoenixDatabaseMetaData.SYSTEM_STATS_NAME
+                    + " WHERE " + PHYSICAL_NAME + "='" + physicalName.getString() + "' AND " + COLUMN_FAMILY
+                    + " IS NULL AND " + LAST_STATS_UPDATE_TIME + " IS NOT NULL";
+            ResultSet rs = connection.createStatement().executeQuery(query);
+
+            if (rs.next()) {
+                msSinceLastUpdate = rs.getLong(1) - rs.getLong(2);
+            }
         }
         long rowCount = 0;
         if (msSinceLastUpdate >= msMinBetweenUpdates) {
@@ -1976,14 +2002,14 @@ public class MetaDataClient {
                 }
             }
 
-            boolean useStatsForParallelization = true;
-            Boolean useStatsForParallelizationProp = (Boolean) TableProperty.USE_STATS_FOR_PARALLELIZATION.getValue(tableProps);
+            boolean useStatsForParallelization =
+                    connection.getQueryServices().getProps().getBoolean(
+                        QueryServices.USE_STATS_FOR_PARALLELIZATION,
+                        QueryServicesOptions.DEFAULT_USE_STATS_FOR_PARALLELIZATION);
+            Boolean useStatsForParallelizationProp =
+                    (Boolean) TableProperty.USE_STATS_FOR_PARALLELIZATION.getValue(tableProps);
             if (useStatsForParallelizationProp != null) {
                 useStatsForParallelization = useStatsForParallelizationProp;
-            } else {
-                useStatsForParallelization = connection.getQueryServices().getProps().getBoolean(
-                    QueryServices.USE_STATS_FOR_PARALLELIZATION,
-                    QueryServicesOptions.DEFAULT_USE_STATS_FOR_PARALLELIZATION);
             }
 
             boolean sharedTable = statement.getTableType() == PTableType.VIEW || allocateIndexId;


[03/20] phoenix git commit: PHOENIX-4329 Test IndexScrutinyTool while table is taking writes (Vincent Poon)

Posted by ja...@apache.org.
PHOENIX-4329 Test IndexScrutinyTool while table is taking writes (Vincent Poon)


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

Branch: refs/heads/5.x-HBase-2.0
Commit: f7b16a99b176ad6c032654928a67c82f23e74870
Parents: c39cd80
Author: James Taylor <jt...@salesforce.com>
Authored: Sun Oct 29 15:20:23 2017 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Thu Nov 9 12:42:08 2017 -0800

----------------------------------------------------------------------
 .../phoenix/end2end/IndexScrutinyToolIT.java    | 101 ++++++++++++++++++-
 1 file changed, 96 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/f7b16a99/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexScrutinyToolIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexScrutinyToolIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexScrutinyToolIT.java
index 10595a7..cbce7b2 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexScrutinyToolIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexScrutinyToolIT.java
@@ -36,6 +36,9 @@ import java.util.Properties;
 import java.util.Random;
 import java.util.TreeSet;
 import java.util.UUID;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
 
 import com.google.common.collect.Sets;
 import org.apache.commons.io.IOUtils;
@@ -43,6 +46,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.mapreduce.Counters;
 import org.apache.hadoop.mapreduce.Job;
@@ -103,6 +107,7 @@ public class IndexScrutinyToolIT extends BaseTest {
     private PreparedStatement indexTableUpsertStmt;
 
     private long testTime;
+    private Properties props;
 
     @Parameterized.Parameters
     public static Collection<Object[]> data() {
@@ -120,8 +125,11 @@ public class IndexScrutinyToolIT extends BaseTest {
 
     @BeforeClass
     public static void doSetup() throws Exception {
-        Map<String, String> props = Maps.newHashMap();
-        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+        Map<String, String> serverProps = Maps.newHashMap();
+        //disable major compactions
+        serverProps.put(HConstants.MAJOR_COMPACTION_PERIOD, "0");
+        Map<String, String> clientProps = Maps.newHashMap();
+        setUpTestDriver(new ReadOnlyProps(serverProps.entrySet().iterator()), new ReadOnlyProps(clientProps.entrySet().iterator()));
     }
 
     /**
@@ -133,7 +141,7 @@ public class IndexScrutinyToolIT extends BaseTest {
         createTestTable(getUrl(), String.format(dataTableDdl, dataTableFullName));
         createTestTable(getUrl(),
             String.format(indexTableDdl, indexTableName, dataTableFullName));
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         conn = DriverManager.getConnection(getUrl(), props);
         String dataTableUpsert = String.format(UPSERT_SQL, dataTableFullName);
         dataTableUpsertStmt = conn.prepareStatement(dataTableUpsert);
@@ -141,6 +149,7 @@ public class IndexScrutinyToolIT extends BaseTest {
         indexTableUpsertStmt = conn.prepareStatement(indexTableUpsert);
         conn.setAutoCommit(false);
         testTime = EnvironmentEdgeManager.currentTimeMillis() - 1000;
+
     }
 
     @After
@@ -177,6 +186,77 @@ public class IndexScrutinyToolIT extends BaseTest {
     }
 
     /**
+     * Tests running a scrutiny while updates and deletes are happening.
+     * Since CURRENT_SCN is set, the scrutiny shouldn't report any issue.
+     */
+    @Test
+    public void testScrutinyWhileTakingWrites() throws Exception {
+        int id = 0;
+        while (id < 1000) {
+            int index = 1;
+            dataTableUpsertStmt.setInt(index++, id);
+            dataTableUpsertStmt.setString(index++, "name-" + id);
+            dataTableUpsertStmt.setInt(index++, id);
+            dataTableUpsertStmt.setTimestamp(index++, new Timestamp(testTime));
+            dataTableUpsertStmt.executeUpdate();
+            id++;
+        }
+        conn.commit();
+
+        //CURRENT_SCN for scrutiny
+        long scrutinyTS = EnvironmentEdgeManager.currentTimeMillis();
+
+        // launch background upserts and deletes
+        final Random random = new Random(0);
+        Runnable backgroundUpserts = new Runnable() {
+            @Override
+            public void run() {
+                int idToUpsert = random.nextInt(1000);
+                try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+                    PreparedStatement dataPS =
+                            conn.prepareStatement(String.format(UPSERT_SQL, dataTableFullName));
+                    upsertRow(dataPS, idToUpsert, "modified-" + idToUpsert, idToUpsert + 1000);
+                    conn.commit();
+                } catch (SQLException e) {
+                    e.printStackTrace();
+                }
+            }
+        };
+        Runnable backgroundDeletes = new Runnable() {
+            @Override
+            public void run() {
+                int idToDelete = random.nextInt(1000);
+                try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+                    String deleteSql =
+                            String.format(DELETE_SQL, indexTableFullName) + "WHERE \":ID\"="
+                                    + idToDelete;
+                    conn.createStatement().executeUpdate(deleteSql);
+                    conn.commit();
+                } catch (SQLException e) {
+                    e.printStackTrace();
+                }
+            }
+        };
+        ScheduledExecutorService scheduledThreadPool = Executors.newScheduledThreadPool(2);
+        scheduledThreadPool.scheduleWithFixedDelay(backgroundUpserts, 200, 200,
+            TimeUnit.MILLISECONDS);
+        scheduledThreadPool.scheduleWithFixedDelay(backgroundDeletes, 200, 200,
+            TimeUnit.MILLISECONDS);
+
+        // scrutiny should report everything as ok
+        List<Job> completedJobs =
+                runScrutinyCurrentSCN(schemaName, dataTableName, indexTableName,
+                    scrutinyTS);
+        Job job = completedJobs.get(0);
+        assertTrue(job.isSuccessful());
+        Counters counters = job.getCounters();
+        assertEquals(1000, getCounterValue(counters, VALID_ROW_COUNT));
+        assertEquals(0, getCounterValue(counters, INVALID_ROW_COUNT));
+        scheduledThreadPool.shutdown();
+        scheduledThreadPool.awaitTermination(10000, TimeUnit.MILLISECONDS);
+    }
+
+    /**
      * Tests an index with the same # of rows as the data table, but one of the index rows is
      * incorrect Scrutiny should report the invalid rows.
      */
@@ -570,6 +650,13 @@ public class IndexScrutinyToolIT extends BaseTest {
     private String[] getArgValues(String schemaName, String dataTable, String indxTable, Long batchSize,
             SourceTable sourceTable, boolean outputInvalidRows, OutputFormat outputFormat,
             Long maxOutputRows) {
+        return getArgValues(schemaName, dataTable, indxTable, batchSize, sourceTable,
+            outputInvalidRows, outputFormat, maxOutputRows, Long.MAX_VALUE);
+    }
+
+    private String[] getArgValues(String schemaName, String dataTable, String indxTable, Long batchSize,
+            SourceTable sourceTable, boolean outputInvalidRows, OutputFormat outputFormat,
+            Long maxOutputRows, Long scrutinyTs) {
         final List<String> args = Lists.newArrayList();
         if (schemaName != null) {
             args.add("-s");
@@ -591,7 +678,7 @@ public class IndexScrutinyToolIT extends BaseTest {
             args.add(outputDir);
         }
         args.add("-t");
-        args.add(String.valueOf(Long.MAX_VALUE));
+        args.add(String.valueOf(scrutinyTs));
         args.add("-run-foreground");
         if (batchSize != null) {
             args.add("-b");
@@ -619,6 +706,10 @@ public class IndexScrutinyToolIT extends BaseTest {
         return args.toArray(new String[0]);
     }
 
+    private List<Job> runScrutinyCurrentSCN(String schemaName, String dataTableName, String indexTableName, Long scrutinyTS) throws Exception {
+        return runScrutiny(getArgValues(schemaName, dataTableName, indexTableName, null, SourceTable.BOTH, false, null, null, scrutinyTS));
+    }
+
     private List<Job> runScrutiny(String schemaName, String dataTableName, String indexTableName) throws Exception {
         return runScrutiny(schemaName, dataTableName, indexTableName, null, null);
     }
@@ -632,7 +723,7 @@ public class IndexScrutinyToolIT extends BaseTest {
             Long batchSize, SourceTable sourceTable) throws Exception {
         final String[] cmdArgs =
                 getArgValues(schemaName, dataTableName, indexTableName, batchSize, sourceTable, false,
-                    null, null);
+                    null, null, Long.MAX_VALUE);
         return runScrutiny(cmdArgs);
     }
 


[13/20] phoenix git commit: PHOENIX-4287 Make indexes inherit use stats property from their parent table or view

Posted by ja...@apache.org.
PHOENIX-4287 Make indexes inherit use stats property from their parent table or view


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

Branch: refs/heads/5.x-HBase-2.0
Commit: 072ff5857f587c64472c3e03d348321b01cde396
Parents: 59e49f7
Author: Samarth Jain <sa...@apache.org>
Authored: Thu Nov 2 16:55:55 2017 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Thu Nov 9 12:52:00 2017 -0800

----------------------------------------------------------------------
 .../end2end/ExplainPlanWithStatsEnabledIT.java  | 146 +++++++++++++++++--
 .../phoenix/iterate/BaseResultIterators.java    |  41 +++++-
 2 files changed, 171 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/072ff585/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
index e76b147..bfc6819 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
@@ -72,8 +72,8 @@ public class ExplainPlanWithStatsEnabledIT extends ParallelStatsEnabledIT {
     private static void createIndex(String indexName, String table, long guidePostWidth)
             throws Exception {
         try (Connection conn = DriverManager.getConnection(getUrl())) {
-            conn.createStatement().execute("CREATE INDEX " + indexName + " ON " + table
-                    + " (c1.a) INCLUDE (c2.b) ");
+            conn.createStatement().execute(
+                "CREATE INDEX " + indexName + " ON " + table + " (c1.a) INCLUDE (c2.b) ");
             conn.createStatement().execute("UPDATE STATISTICS " + indexName);
         }
     }
@@ -558,9 +558,10 @@ public class ExplainPlanWithStatsEnabledIT extends ParallelStatsEnabledIT {
             Estimate info = getByteRowEstimates(conn, sql, binds);
             assertEquals((Long) 10l, info.getEstimatedRows());
             assertTrue(info.getEstimateInfoTs() > 0);
-            
+
             // Now, let's disable USE_STATS_FOR_PARALLELIZATION on the table
-            conn.createStatement().execute("ALTER TABLE " + tableName + " SET USE_STATS_FOR_PARALLELIZATION = " + false);
+            conn.createStatement().execute(
+                "ALTER TABLE " + tableName + " SET USE_STATS_FOR_PARALLELIZATION = " + false);
             rs = conn.createStatement().executeQuery(sql);
             // stats are not being used for parallelization. So number of scans is lower.
             assertEquals(4, rs.unwrap(PhoenixResultSet.class).getStatement().getQueryPlan()
@@ -570,11 +571,11 @@ public class ExplainPlanWithStatsEnabledIT extends ParallelStatsEnabledIT {
             info = getByteRowEstimates(conn, sql, binds);
             assertEquals((Long) 10l, info.getEstimatedRows());
             assertTrue(info.getEstimateInfoTs() > 0);
-            
+
             // assert that the aggregate query on view also works correctly
             String viewName = "V_" + generateUniqueName();
-            conn.createStatement()
-                    .execute("CREATE VIEW " + viewName + " AS SELECT * FROM " + tableName + " USE_STATS_FOR_PARALLELIZATION = false");
+            conn.createStatement().execute("CREATE VIEW " + viewName + " AS SELECT * FROM "
+                    + tableName + " USE_STATS_FOR_PARALLELIZATION = false");
             sql = "SELECT COUNT(*) FROM " + viewName;
             rs = conn.createStatement().executeQuery(sql);
             // stats are not being used for parallelization. So number of scans is lower.
@@ -595,21 +596,21 @@ public class ExplainPlanWithStatsEnabledIT extends ParallelStatsEnabledIT {
             rs = conn.createStatement().executeQuery(sql);
             // stats are being used for parallelization. So number of scans is higher.
             assertEquals(14, rs.unwrap(PhoenixResultSet.class).getStatement().getQueryPlan()
-                .getScans().get(0).size());
+                    .getScans().get(0).size());
             assertTrue(rs.next());
             assertEquals(10, rs.getInt(1));
             info = getByteRowEstimates(conn, sql, binds);
             assertEquals((Long) 10l, info.getEstimatedRows());
             assertTrue(info.getEstimateInfoTs() > 0);
 
-            conn.createStatement().execute(
-                "ALTER TABLE " + viewName + " SET USE_STATS_FOR_PARALLELIZATION=true");
+            conn.createStatement()
+                    .execute("ALTER TABLE " + viewName + " SET USE_STATS_FOR_PARALLELIZATION=true");
             sql = "SELECT COUNT(*) FROM " + viewName;
             // query the view
             rs = conn.createStatement().executeQuery(sql);
             // stats are not being used for parallelization. So number of scans is higher.
             assertEquals(14, rs.unwrap(PhoenixResultSet.class).getStatement().getQueryPlan()
-                .getScans().get(0).size());
+                    .getScans().get(0).size());
             assertTrue(rs.next());
             assertEquals(10, rs.getInt(1));
             info = getByteRowEstimates(conn, sql, binds);
@@ -944,4 +945,127 @@ public class ExplainPlanWithStatsEnabledIT extends ParallelStatsEnabledIT {
             assertEquals((Long) 6l, info.estimatedRows);
         }
     }
+
+    @Test
+    public void testIndexesUseStatsIfOnForParentTable() throws Exception {
+        testIndexesInheritUseStatsPropFromParentTable(true);
+    }
+
+    @Test
+    public void testIndexesDontUseStatsIfOffForParentTable() throws Exception {
+        testIndexesInheritUseStatsPropFromParentTable(false);
+    }
+
+    private void testIndexesInheritUseStatsPropFromParentTable(boolean useStats) throws Exception {
+        String baseTable = generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String ddl =
+                    "CREATE TABLE " + baseTable
+                            + " (k INTEGER PRIMARY KEY, a bigint, b bigint, c bigint) GUIDE_POSTS_WIDTH=20, USE_STATS_FOR_PARALLELIZATION="
+                            + useStats;
+            conn.createStatement().execute(ddl);
+            conn.createStatement().execute("upsert into " + baseTable + " values (100,1,1,1)");
+            conn.createStatement().execute("upsert into " + baseTable + " values (101,2,2,2)");
+            conn.createStatement().execute("upsert into " + baseTable + " values (102,3,3,3)");
+            conn.createStatement().execute("upsert into " + baseTable + " values (103,4,4,4)");
+            conn.createStatement().execute("upsert into " + baseTable + " values (104,5,5,5)");
+            conn.createStatement().execute("upsert into " + baseTable + " values (105,6,6,6)");
+            conn.createStatement().execute("upsert into " + baseTable + " values (106,7,7,7)");
+            conn.createStatement().execute("upsert into " + baseTable + " values (107,8,8,8)");
+            conn.createStatement().execute("upsert into " + baseTable + " values (108,9,9,9)");
+            conn.createStatement().execute("upsert into " + baseTable + " values (109,10,10,10)");
+            conn.commit();
+
+            // Create global index on base table
+            String globalIndex = "GI_" + generateUniqueName();
+            ddl = "CREATE INDEX " + globalIndex + " ON " + baseTable + " (a) INCLUDE (b) ";
+            conn.createStatement().execute(ddl);
+
+            // Create local index on base table
+            String localIndex = "LI_" + generateUniqueName();
+            ddl = "CREATE LOCAL INDEX " + localIndex + " ON " + baseTable + " (b) INCLUDE (c) ";
+            conn.createStatement().execute(ddl);
+
+            // Create a view and an index on it
+            String view = "V_" + generateUniqueName();
+            ddl =
+                    "CREATE VIEW " + view + " AS SELECT * FROM " + baseTable
+                            + " USE_STATS_FOR_PARALLELIZATION=" + useStats;
+            conn.createStatement().execute(ddl);
+            String viewIndex = "VI_" + generateUniqueName();
+            ddl = "CREATE INDEX " + viewIndex + " ON " + view + " (b)";
+            conn.createStatement().execute(ddl);
+
+            // collect stats for all
+            conn.createStatement().execute("UPDATE STATISTICS " + baseTable);
+
+            // query against the base table
+            String query = "SELECT /*+ NO_INDEX */ COUNT(*) FROM " + baseTable;
+            PhoenixResultSet rs =
+                    conn.createStatement().executeQuery(query).unwrap(PhoenixResultSet.class);
+            // assert query is against base table
+            assertEquals(baseTable,
+                rs.getStatement().getQueryPlan().getTableRef().getTable().getName().getString());
+            assertEquals(useStats ? 11 : 1, rs.unwrap(PhoenixResultSet.class).getStatement()
+                    .getQueryPlan().getScans().get(0).size());
+
+            // query against the global index
+            query = "SELECT B FROM " + baseTable + " WHERE A > 0";
+            rs = conn.createStatement().executeQuery(query).unwrap(PhoenixResultSet.class);
+            // assert query is against global index
+            assertEquals(globalIndex, rs.unwrap(PhoenixResultSet.class).getStatement()
+                    .getQueryPlan().getTableRef().getTable().getName().getString());
+            assertEquals(useStats ? 11 : 1, rs.unwrap(PhoenixResultSet.class).getStatement()
+                    .getQueryPlan().getScans().get(0).size());
+
+            // query against the local index
+            query = "SELECT C FROM " + baseTable + " WHERE B > 0";
+            rs = conn.createStatement().executeQuery(query).unwrap(PhoenixResultSet.class);
+            // assert query is against global index
+            assertEquals(localIndex, rs.unwrap(PhoenixResultSet.class).getStatement().getQueryPlan()
+                    .getTableRef().getTable().getName().getString());
+            assertEquals(useStats ? 11 : 1, rs.unwrap(PhoenixResultSet.class).getStatement()
+                    .getQueryPlan().getScans().get(0).size());
+
+            // query against the view
+            query = "SELECT * FROM " + view;
+            rs = conn.createStatement().executeQuery(query).unwrap(PhoenixResultSet.class);
+            // assert query is against view
+            assertEquals(view, rs.unwrap(PhoenixResultSet.class).getStatement().getQueryPlan()
+                    .getTableRef().getTable().getName().getString());
+            assertEquals(useStats ? 11 : 1, rs.unwrap(PhoenixResultSet.class).getStatement()
+                    .getQueryPlan().getScans().get(0).size());
+
+            // query against the view index
+            query = "SELECT 1 FROM " + view + " WHERE B > 0";
+            rs = conn.createStatement().executeQuery(query).unwrap(PhoenixResultSet.class);
+            // assert query is against viewIndex
+            assertEquals(viewIndex, rs.unwrap(PhoenixResultSet.class).getStatement().getQueryPlan()
+                    .getTableRef().getTable().getName().getString());
+            assertEquals(useStats ? 11 : 1, rs.unwrap(PhoenixResultSet.class).getStatement()
+                    .getQueryPlan().getScans().get(0).size());
+
+            // flip the use stats property on the view and see if view index picks it up
+            conn.createStatement().execute(
+                "ALTER VIEW " + view + " SET USE_STATS_FOR_PARALLELIZATION=" + !useStats);
+
+            // query against the view
+            query = "SELECT * FROM " + view;
+            rs = conn.createStatement().executeQuery(query).unwrap(PhoenixResultSet.class);
+            // assert query is against view
+            assertEquals(view, rs.unwrap(PhoenixResultSet.class).getStatement().getQueryPlan()
+                    .getTableRef().getTable().getName().getString());
+            assertEquals(!useStats ? 11 : 1, rs.unwrap(PhoenixResultSet.class).getStatement()
+                    .getQueryPlan().getScans().get(0).size());
+
+            // query against the view index
+            query = "SELECT 1 FROM " + view + " WHERE B > 0";
+            rs = conn.createStatement().executeQuery(query).unwrap(PhoenixResultSet.class);
+            // assert query is against viewIndex
+            assertEquals(viewIndex, rs.unwrap(PhoenixResultSet.class).getStatement().getQueryPlan()
+                    .getTableRef().getTable().getName().getString());
+            assertEquals(!useStats ? 11 : 1, rs.unwrap(PhoenixResultSet.class).getStatement()
+                    .getQueryPlan().getScans().get(0).size());
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/072ff585/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
index 72080be..9010a64 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
@@ -82,6 +82,7 @@ import org.apache.phoenix.filter.DistinctPrefixFilter;
 import org.apache.phoenix.filter.EncodedQualifiersColumnProjectionFilter;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.hbase.index.util.VersionUtil;
+import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.join.HashCacheClient;
 import org.apache.phoenix.parse.FilterableStatement;
 import org.apache.phoenix.parse.HintNode;
@@ -98,7 +99,10 @@ import org.apache.phoenix.schema.PTable.ImmutableStorageScheme;
 import org.apache.phoenix.schema.PTable.IndexType;
 import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
 import org.apache.phoenix.schema.PTable.ViewType;
+import org.apache.phoenix.schema.PTableKey;
+import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.StaleRegionBoundaryCacheException;
+import org.apache.phoenix.schema.TableNotFoundException;
 import org.apache.phoenix.schema.TableRef;
 import org.apache.phoenix.schema.stats.GuidePostsInfo;
 import org.apache.phoenix.schema.stats.GuidePostsKey;
@@ -491,11 +495,7 @@ public abstract class BaseResultIterators extends ExplainTable implements Result
         scanId = new UUID(ThreadLocalRandom.current().nextLong(), ThreadLocalRandom.current().nextLong()).toString();
         
         initializeScan(plan, perScanLimit, offset, scan);
-        this.useStatsForParallelization =
-                table.useStatsForParallelization() == null
-                        ? context.getConnection().getQueryServices().getConfiguration().getBoolean(
-                            USE_STATS_FOR_PARALLELIZATION, DEFAULT_USE_STATS_FOR_PARALLELIZATION)
-                        : table.useStatsForParallelization();
+        this.useStatsForParallelization = getStatsForParallelizationProp(context, table);
         this.scans = getParallelScans();
         List<KeyRange> splitRanges = Lists.newArrayListWithExpectedSize(scans.size() * ESTIMATED_GUIDEPOSTS_PER_REGION);
         for (List<Scan> scanList : scans) {
@@ -1238,4 +1238,35 @@ public abstract class BaseResultIterators extends ExplainTable implements Result
     public Long getEstimateInfoTimestamp() {
         return this.estimateInfoTimestamp;
     }
+
+    private boolean getStatsForParallelizationProp(StatementContext context, PTable table) {
+        Boolean useStats = table.useStatsForParallelization();
+        if (useStats != null) {
+            return useStats;
+        }
+        /*
+         * For a view index, we use the property set on view. For indexes on base table, whether
+         * global or local, we use the property set on the base table.
+         */
+        if (table.getType() == PTableType.INDEX) {
+            PhoenixConnection conn = context.getConnection();
+            String parentTableName = table.getParentName().getString();
+            try {
+                PTable parentTable =
+                        conn.getTable(new PTableKey(conn.getTenantId(), parentTableName));
+                useStats = parentTable.useStatsForParallelization();
+                if (useStats != null) {
+                    return useStats;
+                }
+            } catch (TableNotFoundException e) {
+                logger.warn("Unable to find parent table \"" + parentTableName + "\" of table \""
+                        + table.getName().getString()
+                        + "\" to determine USE_STATS_FOR_PARALLELIZATION",
+                    e);
+            }
+        }
+        return context.getConnection().getQueryServices().getConfiguration()
+                .getBoolean(USE_STATS_FOR_PARALLELIZATION, DEFAULT_USE_STATS_FOR_PARALLELIZATION);
+    }
+
 }


[05/20] phoenix git commit: PHOENIX-4290 Full table scan performed for DELETE with table having immutable indexes

Posted by ja...@apache.org.
PHOENIX-4290 Full table scan performed for DELETE with table having immutable indexes


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

Branch: refs/heads/5.x-HBase-2.0
Commit: 7a5b5da589372e0d25eec7aa66c5c513e73eab04
Parents: f7b16a9
Author: James Taylor <jt...@salesforce.com>
Authored: Mon Oct 30 19:25:53 2017 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Thu Nov 9 12:42:32 2017 -0800

----------------------------------------------------------------------
 .../org/apache/phoenix/end2end/DeleteIT.java    | 134 ++-
 .../phoenix/end2end/index/ImmutableIndexIT.java |  22 +-
 .../end2end/index/IndexMaintenanceIT.java       |  18 +-
 .../org/apache/phoenix/tx/TxCheckpointIT.java   |  18 +-
 .../apache/phoenix/compile/DeleteCompiler.java  | 849 ++++++++++---------
 .../apache/phoenix/compile/FromCompiler.java    |  49 +-
 .../compile/TupleProjectionCompiler.java        |   2 +-
 .../phoenix/exception/SQLExceptionCode.java     |   1 -
 .../apache/phoenix/execute/MutationState.java   |   4 +-
 .../apache/phoenix/index/IndexMaintainer.java   |  35 +-
 .../apache/phoenix/optimize/QueryOptimizer.java |   2 +-
 .../org/apache/phoenix/schema/PTableImpl.java   |  10 +
 .../java/org/apache/phoenix/util/IndexUtil.java |  18 +-
 .../phoenix/compile/QueryCompilerTest.java      |  27 -
 14 files changed, 643 insertions(+), 546 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/7a5b5da5/phoenix-core/src/it/java/org/apache/phoenix/end2end/DeleteIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DeleteIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DeleteIT.java
index 09e1021..aa4d36e 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DeleteIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DeleteIT.java
@@ -19,7 +19,6 @@ package org.apache.phoenix.end2end;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
 
 import java.sql.Connection;
 import java.sql.Date;
@@ -33,7 +32,10 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.util.QueryUtil;
+import org.apache.phoenix.util.TestUtil;
 import org.junit.Test;
 
 
@@ -136,18 +138,25 @@ public class DeleteIT extends ParallelStatsDisabledIT {
         rs.close();
     }
     
-    private static void assertIndexUsed (Connection conn, String query, String indexName, boolean expectedToBeUsed) throws SQLException {
-        assertIndexUsed(conn, query, Collections.emptyList(), indexName, expectedToBeUsed);
+    private static void assertIndexUsed (Connection conn, String query, String indexName, boolean expectedToBeUsed, boolean local) throws SQLException {
+        assertIndexUsed(conn, query, Collections.emptyList(), indexName, expectedToBeUsed, local);
     }
 
-    private static void assertIndexUsed (Connection conn, String query, List<Object> binds, String indexName, boolean expectedToBeUsed) throws SQLException {
+    private static void assertIndexUsed (Connection conn, String query, List<Object> binds, String indexName, boolean expectedToBeUsed, boolean local) throws SQLException {
             PreparedStatement stmt = conn.prepareStatement("EXPLAIN " + query);
             for (int i = 0; i < binds.size(); i++) {
                 stmt.setObject(i+1, binds.get(i));
             }
             ResultSet rs = stmt.executeQuery();
             String explainPlan = QueryUtil.getExplainPlan(rs);
-            assertEquals(expectedToBeUsed, explainPlan.contains(" SCAN OVER " + indexName));
+            // It's very difficult currently to check if a local index is being used
+            // This check is brittle as it checks that the index ID appears in the range scan
+            // TODO: surface QueryPlan from MutationPlan
+            if (local) {
+                assertEquals(expectedToBeUsed, explainPlan.contains(indexName + " [1]") || explainPlan.contains(indexName + " [1,"));
+            } else {
+                assertEquals(expectedToBeUsed, explainPlan.contains(" SCAN OVER " + indexName));
+            }
    }
 
     private void testDeleteRange(boolean autoCommit, boolean createIndex) throws Exception {
@@ -190,9 +199,7 @@ public class DeleteIT extends ParallelStatsDisabledIT {
         PreparedStatement stmt;
         conn.setAutoCommit(autoCommit);
         deleteStmt = "DELETE FROM " + tableName + " WHERE i >= ? and i < ?";
-        if(!local) {
-            assertIndexUsed(conn, deleteStmt, Arrays.<Object>asList(5,10), indexInUse, false);
-        }
+        assertIndexUsed(conn, deleteStmt, Arrays.<Object>asList(5,10), indexInUse, false, local);
         stmt = conn.prepareStatement(deleteStmt);
         stmt.setInt(1, 5);
         stmt.setInt(2, 10);
@@ -202,7 +209,7 @@ public class DeleteIT extends ParallelStatsDisabledIT {
         }
         
         String query = "SELECT count(*) FROM " + tableName;
-        assertIndexUsed(conn, query, indexInUse, createIndex);
+        assertIndexUsed(conn, query, indexInUse, createIndex, local);
         query = "SELECT count(*) FROM " + tableName;
         rs = conn.createStatement().executeQuery(query);
         assertTrue(rs.next());
@@ -210,9 +217,7 @@ public class DeleteIT extends ParallelStatsDisabledIT {
         
         deleteStmt = "DELETE FROM " + tableName + " WHERE j IS NULL";
         stmt = conn.prepareStatement(deleteStmt);
-        if(!local) {
-            assertIndexUsed(conn, deleteStmt, indexInUse, createIndex);
-        }
+        assertIndexUsed(conn, deleteStmt, indexInUse, createIndex, local);
         int deleteCount = stmt.executeUpdate();
         assertEquals(3, deleteCount);
         if (!autoCommit) {
@@ -254,40 +259,40 @@ public class DeleteIT extends ParallelStatsDisabledIT {
     }
 
     @Test
-    public void testDeleteAllFromTableWithIndexAutoCommitSalting() throws SQLException {
+    public void testDeleteAllFromTableWithIndexAutoCommitSalting() throws Exception {
         testDeleteAllFromTableWithIndex(true, true, false);
     }
 
     @Test
-    public void testDeleteAllFromTableWithLocalIndexAutoCommitSalting() throws SQLException {
+    public void testDeleteAllFromTableWithLocalIndexAutoCommitSalting() throws Exception {
         testDeleteAllFromTableWithIndex(true, true, true);
     }
     
     @Test
-    public void testDeleteAllFromTableWithIndexAutoCommitNoSalting() throws SQLException {
+    public void testDeleteAllFromTableWithIndexAutoCommitNoSalting() throws Exception {
         testDeleteAllFromTableWithIndex(true, false);
     }
     
     @Test
-    public void testDeleteAllFromTableWithIndexNoAutoCommitNoSalting() throws SQLException {
+    public void testDeleteAllFromTableWithIndexNoAutoCommitNoSalting() throws Exception {
         testDeleteAllFromTableWithIndex(false,false);
     }
     
     @Test
-    public void testDeleteAllFromTableWithIndexNoAutoCommitSalted() throws SQLException {
+    public void testDeleteAllFromTableWithIndexNoAutoCommitSalted() throws Exception {
         testDeleteAllFromTableWithIndex(false, true, false);
     }
     
     @Test
-    public void testDeleteAllFromTableWithLocalIndexNoAutoCommitSalted() throws SQLException {
+    public void testDeleteAllFromTableWithLocalIndexNoAutoCommitSalted() throws Exception {
         testDeleteAllFromTableWithIndex(false, true, true);
     }
 
-    private void testDeleteAllFromTableWithIndex(boolean autoCommit, boolean isSalted) throws SQLException {
+    private void testDeleteAllFromTableWithIndex(boolean autoCommit, boolean isSalted) throws Exception {
         testDeleteAllFromTableWithIndex(autoCommit, isSalted, false);
     }
 
-    private void testDeleteAllFromTableWithIndex(boolean autoCommit, boolean isSalted, boolean localIndex) throws SQLException {
+    private void testDeleteAllFromTableWithIndex(boolean autoCommit, boolean isSalted, boolean localIndex) throws Exception {
         Connection con = null;
         try {
             con = DriverManager.getConnection(getUrl());
@@ -334,6 +339,8 @@ public class DeleteIT extends ParallelStatsDisabledIT {
                 con.commit();
             }
             
+            TestUtil.dumpTable(con.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes(tableName)));
+            
             ResultSet rs = con.createStatement().executeQuery("SELECT /*+ NO_INDEX */ count(*) FROM " + tableName);
             assertTrue(rs.next());
             assertEquals(0, rs.getLong(1));
@@ -354,16 +361,16 @@ public class DeleteIT extends ParallelStatsDisabledIT {
     }
     
     @Test
-    public void testDeleteRowFromTableWithImmutableIndex() throws SQLException {
-        testDeleteRowFromTableWithImmutableIndex(false);
+    public void testDeleteRowFromTableWithImmutableIndex() throws Exception {
+        testDeleteRowFromTableWithImmutableIndex(false, true);
     }
     
     @Test
-    public void testDeleteRowFromTableWithImmutableLocalIndex() throws SQLException {
-        testDeleteRowFromTableWithImmutableIndex(true);
+    public void testDeleteRowFromTableWithImmutableLocalIndex() throws Exception {
+        testDeleteRowFromTableWithImmutableIndex(true, false);
     }
     
-    public void testDeleteRowFromTableWithImmutableIndex(boolean localIndex) throws SQLException {
+    public void testDeleteRowFromTableWithImmutableIndex(boolean localIndex, boolean useCoveredIndex) throws Exception {
         Connection con = null;
         try {
             boolean autoCommit = false;
@@ -375,6 +382,7 @@ public class DeleteIT extends ParallelStatsDisabledIT {
             String tableName = generateUniqueName();
             String indexName1 = generateUniqueName();
             String indexName2 = generateUniqueName();
+            String indexName3 = useCoveredIndex? generateUniqueName() : null;
 
             stm.execute("CREATE TABLE IF NOT EXISTS " + tableName + " (" +
                     "HOST CHAR(2) NOT NULL," +
@@ -387,6 +395,9 @@ public class DeleteIT extends ParallelStatsDisabledIT {
                     "CONSTRAINT PK PRIMARY KEY (HOST, DOMAIN, FEATURE, \"DATE\")) IMMUTABLE_ROWS=true");
             stm.execute("CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName1 + " ON " + tableName + " (\"DATE\", FEATURE)");
             stm.execute("CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName2 + " ON " + tableName + " (\"DATE\", FEATURE, USAGE.DB)");
+            if (useCoveredIndex) {
+                stm.execute("CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName3 + " ON " + tableName + " (STATS.ACTIVE_VISITOR) INCLUDE (USAGE.CORE, USAGE.DB)");
+            }
             stm.close();
 
             Date date = new Date(0);
@@ -400,39 +411,48 @@ public class DeleteIT extends ParallelStatsDisabledIT {
             psInsert.setLong(6, 2L);
             psInsert.setLong(7, 3);
             psInsert.execute();
-            psInsert.close();
             if (!autoCommit) {
                 con.commit();
             }
             
-            psInsert = con.prepareStatement("DELETE FROM " + tableName + " WHERE (HOST, DOMAIN, FEATURE, \"DATE\") = (?,?,?,?)");
-            psInsert.setString(1, "AA");
-            psInsert.setString(2, "BB");
-            psInsert.setString(3, "CC");
-            psInsert.setDate(4, date);
-            psInsert.execute();
+            PreparedStatement psDelete = con.prepareStatement("DELETE FROM " + tableName + " WHERE (HOST, DOMAIN, FEATURE, \"DATE\") = (?,?,?,?)");
+            psDelete.setString(1, "AA");
+            psDelete.setString(2, "BB");
+            psDelete.setString(3, "CC");
+            psDelete.setDate(4, date);
+            psDelete.execute();
             if (!autoCommit) {
                 con.commit();
             }
             
-            ResultSet rs = con.createStatement().executeQuery("SELECT /*+ NO_INDEX */ count(*) FROM " + tableName);
-            assertTrue(rs.next());
-            assertEquals(0, rs.getLong(1));
+            assertDeleted(con, tableName, indexName1, indexName2, indexName3);
 
-            rs = con.createStatement().executeQuery("SELECT count(*) FROM " + indexName1);
-            assertTrue(rs.next());
-            assertEquals(0, rs.getLong(1));
+            psInsert.execute();
+            if (!autoCommit) {
+                con.commit();
+            }
 
-            stm.execute("DROP INDEX " + indexName1 + " ON " + tableName);
-            stm.execute("DROP INDEX " + indexName2 + " ON " + tableName);
+            psDelete = con.prepareStatement("DELETE FROM " + tableName + " WHERE  USAGE.DB=2");
+            psDelete.execute();
+            if (!autoCommit) {
+                con.commit();
+            }
+
+            assertDeleted(con, tableName, indexName1, indexName2, indexName3);
+
+            psInsert.execute();
+            if (!autoCommit) {
+                con.commit();
+            }
 
-            stm.execute("CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName1 + " ON " + tableName + " (USAGE.DB)");
-            stm.execute("CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName2 + " ON " + tableName + " (USAGE.DB, \"DATE\")");
-            try{
-                psInsert = con.prepareStatement("DELETE FROM " + tableName + " WHERE  USAGE.DB=2");
-            } catch(Exception e) {
-                fail("There should not be any exception while deleting row");
+            psDelete = con.prepareStatement("DELETE FROM " + tableName + " WHERE  ACTIVE_VISITOR=3");
+            psDelete.execute();
+            if (!autoCommit) {
+                con.commit();
             }
+
+            assertDeleted(con, tableName, indexName1, indexName2, indexName3);
+
         } finally {
             try {
                 con.close();
@@ -440,6 +460,28 @@ public class DeleteIT extends ParallelStatsDisabledIT {
             }
         }
     }
+
+    private static void assertDeleted(Connection con, String tableName, String indexName1, String indexName2, String indexName3)
+            throws SQLException {
+        ResultSet rs;
+        rs = con.createStatement().executeQuery("SELECT /*+ NO_INDEX */ count(*) FROM " + tableName);
+        assertTrue(rs.next());
+        assertEquals(0, rs.getLong(1));
+
+        rs = con.createStatement().executeQuery("SELECT count(*) FROM " + indexName1);
+        assertTrue(rs.next());
+        assertEquals(0, rs.getLong(1));
+
+        rs = con.createStatement().executeQuery("SELECT count(*) FROM " + indexName2);
+        assertTrue(rs.next());
+        assertEquals(0, rs.getLong(1));
+
+        if (indexName3 != null) {
+            rs = con.createStatement().executeQuery("SELECT count(*) FROM " + indexName3);
+            assertTrue(rs.next());
+            assertEquals(0, rs.getLong(1));
+        }
+    }
     
     
     @Test

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7a5b5da5/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java
index 9eb5440..e0398c7 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java
@@ -20,7 +20,6 @@ package org.apache.phoenix.end2end.index;
 import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
 
 import java.sql.Connection;
 import java.sql.DriverManager;
@@ -51,7 +50,6 @@ import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.end2end.BaseUniqueNamesOwnClusterIT;
-import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.query.BaseTest;
 import org.apache.phoenix.query.QueryServices;
@@ -149,18 +147,14 @@ public class ImmutableIndexIT extends BaseUniqueNamesOwnClusterIT {
 
             conn.setAutoCommit(true);
             String dml = "DELETE from " + fullTableName + " WHERE long_col2 = 4";
-            try {
-                conn.createStatement().execute(dml);
-                if (!localIndex) {
-                    fail();
-                }
-            } catch (SQLException e) {
-                if (localIndex) {
-                    throw e;
-                }
-                assertEquals(SQLExceptionCode.INVALID_FILTER_ON_IMMUTABLE_ROWS.getErrorCode(),
-                    e.getErrorCode());
-            }
+            conn.createStatement().execute(dml);
+
+            rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullTableName);
+            assertTrue(rs.next());
+            assertEquals(2, rs.getInt(1));
+            rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullIndexName);
+            assertTrue(rs.next());
+            assertEquals(2, rs.getInt(1));
 
             conn.createStatement().execute("DROP TABLE " + fullTableName);
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7a5b5da5/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMaintenanceIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMaintenanceIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMaintenanceIT.java
index d5895ae..9ff5a35 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMaintenanceIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMaintenanceIT.java
@@ -23,7 +23,6 @@ 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.junit.Assert.fail;
 
 import java.math.BigDecimal;
 import java.sql.Connection;
@@ -36,7 +35,6 @@ import java.util.Properties;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
-import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.util.DateUtil;
 import org.apache.phoenix.util.PropertiesUtil;
@@ -341,22 +339,10 @@ public class IndexMaintenanceIT extends ParallelStatsDisabledIT {
             assertEquals(2, rs.getInt(1));
 
             conn.setAutoCommit(true);
-            String dml = "DELETE from " + fullDataTableName + " WHERE long_col2 = 2";
-            try {
-                conn.createStatement().execute(dml);
-                if (!mutable && !localIndex) {
-                    fail();
-                }
-            } catch (SQLException e) {
-                if (mutable || localIndex) {
-                    throw e;
-                }
-                assertEquals(SQLExceptionCode.INVALID_FILTER_ON_IMMUTABLE_ROWS.getErrorCode(), e.getErrorCode());
-            }
+            conn.createStatement().execute("DELETE from " + fullDataTableName + " WHERE long_col2 = 2");
 
             if (!mutable) {
-                dml = "DELETE from " + fullDataTableName + " WHERE 2*long_col2 = 4";
-                conn.createStatement().execute(dml);
+                conn.createStatement().execute("DELETE from " + fullDataTableName + " WHERE 2*long_col2 = 4");
             }
 
             rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullDataTableName);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7a5b5da5/phoenix-core/src/it/java/org/apache/phoenix/tx/TxCheckpointIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/tx/TxCheckpointIT.java b/phoenix-core/src/it/java/org/apache/phoenix/tx/TxCheckpointIT.java
index 989a97e..bf39dfe 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/tx/TxCheckpointIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/tx/TxCheckpointIT.java
@@ -311,7 +311,6 @@ public class TxCheckpointIT extends ParallelStatsDisabledIT {
         String tableName = "TBL_" + generateUniqueName();
         String indexName = "IDX_" + generateUniqueName();
         String fullTableName = SchemaUtil.getTableName(tableName, tableName);
-		Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
 		ResultSet rs;
 		try (Connection conn = getConnection()) {
 			conn.setAutoCommit(false);
@@ -400,6 +399,23 @@ public class TxCheckpointIT extends ParallelStatsDisabledIT {
             assertTrue(rs.next());
             assertEquals(1,rs.getLong(1));
             assertFalse(rs.next());
+            
+            conn.createStatement().execute("drop index " + indexName + " on " + fullTableName + "1");
+            conn.createStatement().execute("delete from " + fullTableName + "1 where id1=fk1b AND fk1b=id1");
+            conn.createStatement().execute("delete from " + fullTableName + "1 where id1 in (select fk1a from " + fullTableName + "1 join " + fullTableName + "2 on (fk2=id1))");
+            assertEquals(PhoenixVisibilityLevel.SNAPSHOT_EXCLUDE_CURRENT, state.getVisibilityLevel());
+            assertNotEquals(wp, state.getWritePointer()); // Make sure write ptr moved
+    
+            rs = conn.createStatement().executeQuery("select /*+ NO_INDEX */ id1 from " + fullTableName + "1");
+            assertTrue(rs.next());
+            assertEquals(1,rs.getLong(1));
+            assertFalse(rs.next());
+    
+            rs = conn.createStatement().executeQuery("select /*+ INDEX(DEMO IDX) */ id1 from " + fullTableName + "1");
+            assertTrue(rs.next());
+            assertEquals(1,rs.getLong(1));
+            assertFalse(rs.next());
+    
 		}
     }  
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7a5b5da5/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
index eb252d3..73689d5 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
@@ -19,12 +19,11 @@ package org.apache.phoenix.compile;
 import static org.apache.phoenix.execute.MutationState.RowTimestampColInfo.NULL_ROWTIMESTAMP_INFO;
 import static org.apache.phoenix.util.NumberUtil.add;
 
+import java.io.IOException;
 import java.sql.ParameterMetaData;
 import java.sql.SQLException;
-import java.util.Arrays;
-import java.util.Collection;
+import java.util.ArrayList;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -34,19 +33,20 @@ import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.cache.ServerCacheClient;
 import org.apache.phoenix.cache.ServerCacheClient.ServerCache;
 import org.apache.phoenix.compile.GroupByCompiler.GroupBy;
 import org.apache.phoenix.compile.OrderByCompiler.OrderBy;
 import org.apache.phoenix.coprocessor.BaseScannerRegionObserver;
-import org.apache.phoenix.coprocessor.MetaDataProtocol.MetaDataMutationResult;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.execute.AggregatePlan;
-import org.apache.phoenix.execute.BaseQueryPlan;
 import org.apache.phoenix.execute.MutationState;
 import org.apache.phoenix.execute.MutationState.RowMutationState;
 import org.apache.phoenix.filter.SkipScanFilter;
+import org.apache.phoenix.hbase.index.ValueGetter;
+import org.apache.phoenix.hbase.index.covered.update.ColumnReference;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.index.IndexMaintainer;
 import org.apache.phoenix.index.PhoenixIndexCodec;
@@ -64,20 +64,20 @@ import org.apache.phoenix.parse.NamedTableNode;
 import org.apache.phoenix.parse.ParseNode;
 import org.apache.phoenix.parse.ParseNodeFactory;
 import org.apache.phoenix.parse.SelectStatement;
+import org.apache.phoenix.parse.TableName;
 import org.apache.phoenix.query.ConnectionQueryServices;
 import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
-import org.apache.phoenix.schema.MetaDataClient;
-import org.apache.phoenix.schema.MetaDataEntityNotFoundException;
+import org.apache.phoenix.schema.DelegateColumn;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PIndexState;
 import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PRow;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTable.IndexType;
-import org.apache.phoenix.schema.PTableKey;
+import org.apache.phoenix.schema.PTableImpl;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.ReadOnlyTableException;
 import org.apache.phoenix.schema.SortOrder;
@@ -105,9 +105,11 @@ public class DeleteCompiler {
         this.operation = operation;
     }
     
-    private static MutationState deleteRows(StatementContext childContext, TableRef targetTableRef, List<TableRef> indexTableRefs, ResultIterator iterator, RowProjector projector, TableRef sourceTableRef) throws SQLException {
-        PTable table = targetTableRef.getTable();
-        PhoenixStatement statement = childContext.getStatement();
+    private static MutationState deleteRows(StatementContext context, ResultIterator iterator, QueryPlan bestPlan, TableRef projectedTableRef, List<TableRef> otherTableRefs) throws SQLException {
+        RowProjector projector = bestPlan.getProjector();
+        TableRef tableRef = bestPlan.getTableRef();
+        PTable table = tableRef.getTable();
+        PhoenixStatement statement = context.getStatement();
         PhoenixConnection connection = statement.getConnection();
         PName tenantId = connection.getTenantId();
         byte[] tenantIdBytes = null;
@@ -123,9 +125,9 @@ public class DeleteCompiler {
         List<Map<ImmutableBytesPtr,RowMutationState>> indexMutations = null;
         // If indexTableRef is set, we're deleting the rows from both the index table and
         // the data table through a single query to save executing an additional one.
-        if (!indexTableRefs.isEmpty()) {
-            indexMutations = Lists.newArrayListWithExpectedSize(indexTableRefs.size());
-            for (int i = 0; i < indexTableRefs.size(); i++) {
+        if (!otherTableRefs.isEmpty()) {
+            indexMutations = Lists.newArrayListWithExpectedSize(otherTableRefs.size());
+            for (int i = 0; i < otherTableRefs.size(); i++) {
                 indexMutations.add(Maps.<ImmutableBytesPtr,RowMutationState>newHashMapWithExpectedSize(batchSize));
             }
         }
@@ -140,38 +142,84 @@ public class DeleteCompiler {
         if (isMultiTenant) {
             values[offset++] = tenantIdBytes;
         }
-        try (PhoenixResultSet rs = new PhoenixResultSet(iterator, projector, childContext)) {
-            int rowCount = 0;
-            while (rs.next()) {
-                ImmutableBytesPtr ptr = new ImmutableBytesPtr();  // allocate new as this is a key in a Map
-                // Use tuple directly, as projector would not have all the PK columns from
-                // our index table inside of our projection. Since the tables are equal,
-                // there's no transation required.
-                if (sourceTableRef.equals(targetTableRef)) {
-                    rs.getCurrentRow().getKey(ptr);
-                } else {
-                    for (int i = offset; i < values.length; i++) {
-                        byte[] byteValue = rs.getBytes(i+1-offset);
-                        // The ResultSet.getBytes() call will have inverted it - we need to invert it back.
-                        // TODO: consider going under the hood and just getting the bytes
-                        if (pkColumns.get(i).getSortOrder() == SortOrder.DESC) {
-                            byte[] tempByteValue = Arrays.copyOf(byteValue, byteValue.length);
-                            byteValue = SortOrder.invert(byteValue, 0, tempByteValue, 0, byteValue.length);
+        try (final PhoenixResultSet rs = new PhoenixResultSet(iterator, projector, context)) {
+            ValueGetter getter = null;
+            if (!otherTableRefs.isEmpty()) {
+                getter = new ValueGetter() {
+                    final ImmutableBytesWritable valuePtr = new ImmutableBytesWritable();
+                    final ImmutableBytesWritable rowKeyPtr = new ImmutableBytesWritable();
+    
+                    @Override
+                    public ImmutableBytesWritable getLatestValue(ColumnReference ref, long ts) throws IOException {
+                        Cell cell = rs.getCurrentRow().getValue(ref.getFamily(), ref.getQualifier());
+                        if (cell == null) {
+                            return null;
                         }
-                        values[i] = byteValue;
+                        valuePtr.set(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
+                        return valuePtr;
+                    }
+    
+                    @Override
+                    public byte[] getRowKey() {
+                        rs.getCurrentRow().getKey(rowKeyPtr);
+                        return ByteUtil.copyKeyBytesIfNecessary(rowKeyPtr);
+                    }
+                };
+            }
+            IndexMaintainer scannedIndexMaintainer = null;
+            IndexMaintainer[] maintainers = null;
+            PTable dataTable = table;
+            if (table.getType() == PTableType.INDEX) {
+                if (!otherTableRefs.isEmpty()) {
+                    // The data table is always the last one in the list if it's
+                    // not chosen as the best of the possible plans.
+                    dataTable = otherTableRefs.get(otherTableRefs.size()-1).getTable();
+                    scannedIndexMaintainer = IndexMaintainer.create(dataTable, table, connection);
+                }
+                maintainers = new IndexMaintainer[otherTableRefs.size()];
+                for (int i = 0; i < otherTableRefs.size(); i++) {
+                    // Create IndexMaintainer based on projected table (i.e. SELECT expressions) so that client-side
+                    // expressions are used instead of server-side ones.
+                    PTable otherTable = otherTableRefs.get(i).getTable();
+                    if (otherTable.getType() == PTableType.INDEX) {
+                        // In this case, we'll convert from index row -> data row -> other index row
+                        maintainers[i] = IndexMaintainer.create(dataTable, otherTable, connection);
+                    } else {
+                        maintainers[i] = scannedIndexMaintainer;
                     }
-                    table.newKey(ptr, values);
                 }
+            } else if (!otherTableRefs.isEmpty()) {
+                dataTable = table;
+                maintainers = new IndexMaintainer[otherTableRefs.size()];
+                for (int i = 0; i < otherTableRefs.size(); i++) {
+                    // Create IndexMaintainer based on projected table (i.e. SELECT expressions) so that client-side
+                    // expressions are used instead of server-side ones.
+                    maintainers[i] = IndexMaintainer.create(projectedTableRef.getTable(), otherTableRefs.get(i).getTable(), connection);
+                }
+
+            }
+            byte[][] viewConstants = IndexUtil.getViewConstants(dataTable);
+            int rowCount = 0;
+            while (rs.next()) {
+                ImmutableBytesPtr rowKeyPtr = new ImmutableBytesPtr();  // allocate new as this is a key in a Map
+                rs.getCurrentRow().getKey(rowKeyPtr);
                 // When issuing deletes, we do not care about the row time ranges. Also, if the table had a row timestamp column, then the
-                // row key will already have its value. 
-                mutations.put(ptr, new RowMutationState(PRow.DELETE_MARKER, statement.getConnection().getStatementExecutionCounter(), NULL_ROWTIMESTAMP_INFO, null));
-                for (int i = 0; i < indexTableRefs.size(); i++) {
+                // row key will already have its value.
+                // Check for otherTableRefs being empty required when deleting directly from the index
+                if (otherTableRefs.isEmpty() || table.getIndexType() != IndexType.LOCAL) {
+                    mutations.put(rowKeyPtr, new RowMutationState(PRow.DELETE_MARKER, statement.getConnection().getStatementExecutionCounter(), NULL_ROWTIMESTAMP_INFO, null));
+                }
+                for (int i = 0; i < otherTableRefs.size(); i++) {
+                    PTable otherTable = otherTableRefs.get(i).getTable();
                     ImmutableBytesPtr indexPtr = new ImmutableBytesPtr(); // allocate new as this is a key in a Map
-                    rs.getCurrentRow().getKey(indexPtr);
                     // Translate the data table row to the index table row
-                    if (sourceTableRef.getTable().getType() != PTableType.INDEX) {
-                        IndexMaintainer maintainer = indexTableRefs.get(i).getTable().getIndexMaintainer(table, connection);
-                        indexPtr.set(maintainer.buildRowKey(null, indexPtr, null, null, HConstants.LATEST_TIMESTAMP));
+                    if (table.getType() == PTableType.INDEX) {
+                        indexPtr.set(scannedIndexMaintainer.buildDataRowKey(rowKeyPtr, viewConstants));
+                        if (otherTable.getType() == PTableType.INDEX) {
+                            indexPtr.set(maintainers[i].buildRowKey(getter, indexPtr, null, null, HConstants.LATEST_TIMESTAMP));                        
+                        }
+                    } else {
+                        indexPtr.set(maintainers[i].buildRowKey(getter, rowKeyPtr, null, null, HConstants.LATEST_TIMESTAMP));
                     }
                     indexMutations.get(i).put(indexPtr, new RowMutationState(PRow.DELETE_MARKER, statement.getConnection().getStatementExecutionCounter(), NULL_ROWTIMESTAMP_INFO, null));
                 }
@@ -181,10 +229,10 @@ public class DeleteCompiler {
                 rowCount++;
                 // Commit a batch if auto commit is true and we're at our batch size
                 if (isAutoCommit && rowCount % batchSize == 0) {
-                    MutationState state = new MutationState(targetTableRef, mutations, 0, maxSize, maxSizeBytes, connection);
+                    MutationState state = new MutationState(tableRef, mutations, 0, maxSize, maxSizeBytes, connection);
                     connection.getMutationState().join(state);
-                    for (int i = 0; i < indexTableRefs.size(); i++) {
-                        MutationState indexState = new MutationState(indexTableRefs.get(i), indexMutations.get(i), 0, maxSize, maxSizeBytes, connection);
+                    for (int i = 0; i < otherTableRefs.size(); i++) {
+                        MutationState indexState = new MutationState(otherTableRefs.get(i), indexMutations.get(i), 0, maxSize, maxSizeBytes, connection);
                         connection.getMutationState().join(indexState);
                     }
                     connection.getMutationState().send();
@@ -197,10 +245,9 @@ public class DeleteCompiler {
 
             // If auto commit is true, this last batch will be committed upon return
             int nCommittedRows = isAutoCommit ? (rowCount / batchSize * batchSize) : 0;
-            MutationState state = new MutationState(targetTableRef, mutations, nCommittedRows, maxSize, maxSizeBytes, connection);
-            for (int i = 0; i < indexTableRefs.size(); i++) {
-                // To prevent the counting of these index rows, we have a negative for remainingRows.
-                MutationState indexState = new MutationState(indexTableRefs.get(i), indexMutations.get(i), 0, maxSize, maxSizeBytes, connection);
+            MutationState state = new MutationState(tableRef, mutations, nCommittedRows, maxSize, maxSizeBytes, connection);
+            for (int i = 0; i < otherTableRefs.size(); i++) {
+                MutationState indexState = new MutationState(otherTableRefs.get(i), indexMutations.get(i), 0, maxSize, maxSizeBytes, connection);
                 state.join(indexState);
             }
             return state;
@@ -208,10 +255,9 @@ public class DeleteCompiler {
     }
     
     private static class DeletingParallelIteratorFactory extends MutatingParallelIteratorFactory {
-        private RowProjector projector;
-        private TableRef targetTableRef;
-        private List<TableRef> indexTableRefs;
-        private TableRef sourceTableRef;
+        private QueryPlan queryPlan;
+        private List<TableRef> otherTableRefs;
+        private TableRef projectedTableRef;
         
         private DeletingParallelIteratorFactory(PhoenixConnection connection) {
             super(connection);
@@ -225,41 +271,36 @@ public class DeleteCompiler {
              * need to be captured are already getting collected in the parent statement context enclosed in the result
              * iterator being used for reading rows out.
              */
-            StatementContext ctx = new StatementContext(statement, false);
-            MutationState state = deleteRows(ctx, targetTableRef, indexTableRefs, iterator, projector, sourceTableRef);
+            StatementContext context = new StatementContext(statement, false);
+            MutationState state = deleteRows(context, iterator, queryPlan, projectedTableRef, otherTableRefs);
             return state;
         }
         
-        public void setTargetTableRef(TableRef tableRef) {
-            this.targetTableRef = tableRef;
+        public void setQueryPlan(QueryPlan queryPlan) {
+            this.queryPlan = queryPlan;
         }
         
-        public void setSourceTableRef(TableRef tableRef) {
-            this.sourceTableRef = tableRef;
+        public void setOtherTableRefs(List<TableRef> otherTableRefs) {
+            this.otherTableRefs = otherTableRefs;
         }
         
-        public void setRowProjector(RowProjector projector) {
-            this.projector = projector;
-        }
-
-        public void setIndexTargetTableRefs(List<TableRef> indexTableRefs) {
-            this.indexTableRefs = indexTableRefs;
+        public void setProjectedTableRef(TableRef projectedTableRef) {
+            this.projectedTableRef = projectedTableRef;
         }
-        
     }
     
-    private Map<PTableKey, PTable> getNonDisabledGlobalImmutableIndexes(TableRef tableRef) {
+    private List<PTable> getNonDisabledGlobalImmutableIndexes(TableRef tableRef) {
         PTable table = tableRef.getTable();
         if (table.isImmutableRows() && !table.getIndexes().isEmpty()) {
-            Map<PTableKey, PTable> nonDisabledIndexes = new HashMap<PTableKey, PTable>(table.getIndexes().size());
+            List<PTable> nonDisabledIndexes = Lists.newArrayListWithExpectedSize(table.getIndexes().size());
             for (PTable index : table.getIndexes()) {
                 if (index.getIndexState() != PIndexState.DISABLE && index.getIndexType() == IndexType.GLOBAL) {
-                    nonDisabledIndexes.put(index.getKey(), index);
+                    nonDisabledIndexes.add(index);
                 }
             }
             return nonDisabledIndexes;
         }
-        return Collections.emptyMap();
+        return Collections.emptyList();
     }
     
     private class MultiDeleteMutationPlan implements MutationPlan {
@@ -361,189 +402,151 @@ public class DeleteCompiler {
         }
     }
 
-    private static boolean hasNonPKIndexedColumns(Collection<PTable> immutableIndexes) {
-        for (PTable index : immutableIndexes) {
-            for (PColumn column : index.getPKColumns()) {
-                if (!IndexUtil.isDataPKColumn(column)) {
-                    return true;
-                }
-            }
-        }
-        return false;
-    }
-    
     public MutationPlan compile(DeleteStatement delete) throws SQLException {
         final PhoenixConnection connection = statement.getConnection();
         final boolean isAutoCommit = connection.getAutoCommit();
-        final boolean hasLimit = delete.getLimit() != null;
+        final boolean hasPostProcessing = delete.getLimit() != null;
         final ConnectionQueryServices services = connection.getQueryServices();
         List<QueryPlan> queryPlans;
         NamedTableNode tableNode = delete.getTable();
         String tableName = tableNode.getName().getTableName();
         String schemaName = tableNode.getName().getSchemaName();
-        boolean retryOnce = !isAutoCommit;
-        TableRef tableRefToBe;
-        boolean noQueryReqd = false;
-        boolean runOnServer = false;
         SelectStatement select = null;
         ColumnResolver resolverToBe = null;
-        Map<PTableKey, PTable> immutableIndex = Collections.emptyMap();
-        DeletingParallelIteratorFactory parallelIteratorFactory;
-        QueryPlan dataPlanToBe = null;
-        while (true) {
-            try {
-                resolverToBe = FromCompiler.getResolverForMutation(delete, connection);
-                tableRefToBe = resolverToBe.getTables().get(0);
-                PTable table = tableRefToBe.getTable();
-                // Cannot update:
-                // - read-only VIEW 
-                // - transactional table with a connection having an SCN
-                // TODO: SchemaUtil.isReadOnly(PTable, connection)?
-                if (table.getType() == PTableType.VIEW && table.getViewType().isReadOnly()) {
-                    throw new ReadOnlyTableException(schemaName,tableName);
-                }
-                else if (table.isTransactional() && connection.getSCN() != null) {
-                   throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_SPECIFY_SCN_FOR_TXN_TABLE).setSchemaName(schemaName)
-                   .setTableName(tableName).build().buildException();
-                }
-                
-                immutableIndex = getNonDisabledGlobalImmutableIndexes(tableRefToBe);
-                boolean mayHaveImmutableIndexes = !immutableIndex.isEmpty();
-                noQueryReqd = !hasLimit;
-                // Can't run on same server for transactional data, as we need the row keys for the data
-                // that is being upserted for conflict detection purposes.
-                runOnServer = isAutoCommit && noQueryReqd && !table.isTransactional();
-                HintNode hint = delete.getHint();
-                if (runOnServer && !delete.getHint().hasHint(Hint.USE_INDEX_OVER_DATA_TABLE)) {
-                    hint = HintNode.create(hint, Hint.USE_DATA_OVER_INDEX_TABLE);
-                }
+        DeletingParallelIteratorFactory parallelIteratorFactoryToBe;
+        resolverToBe = FromCompiler.getResolverForMutation(delete, connection);
+        final TableRef targetTableRef = resolverToBe.getTables().get(0);
+        PTable table = targetTableRef.getTable();
+        // Cannot update:
+        // - read-only VIEW 
+        // - transactional table with a connection having an SCN
+        // TODO: SchemaUtil.isReadOnly(PTable, connection)?
+        if (table.getType() == PTableType.VIEW && table.getViewType().isReadOnly()) {
+            throw new ReadOnlyTableException(schemaName,tableName);
+        }
+        else if (table.isTransactional() && connection.getSCN() != null) {
+           throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_SPECIFY_SCN_FOR_TXN_TABLE).setSchemaName(schemaName)
+           .setTableName(tableName).build().buildException();
+        }
         
-                List<AliasedNode> aliasedNodes = Lists.newArrayListWithExpectedSize(table.getPKColumns().size());
-                boolean isSalted = table.getBucketNum() != null;
-                boolean isMultiTenant = connection.getTenantId() != null && table.isMultiTenant();
-                boolean isSharedViewIndex = table.getViewIndexId() != null;
-                for (int i = (isSalted ? 1 : 0) + (isMultiTenant ? 1 : 0) + (isSharedViewIndex ? 1 : 0); i < table.getPKColumns().size(); i++) {
-                    PColumn column = table.getPKColumns().get(i);
-                    aliasedNodes.add(FACTORY.aliasedNode(null, FACTORY.column(null, '"' + column.getName().getString() + '"', null)));
-                }
-                select = FACTORY.select(delete.getTable(), hint, false, aliasedNodes, delete.getWhere(),
-                        Collections.<ParseNode> emptyList(), null, delete.getOrderBy(), delete.getLimit(), null,
-                        delete.getBindCount(), false, false, Collections.<SelectStatement> emptyList(),
-                        delete.getUdfParseNodes());
-                select = StatementNormalizer.normalize(select, resolverToBe);
-                SelectStatement transformedSelect = SubqueryRewriter.transform(select, resolverToBe, connection);
-                if (transformedSelect != select) {
-                    resolverToBe = FromCompiler.getResolverForQuery(transformedSelect, connection, false, delete.getTable().getName());
-                    select = StatementNormalizer.normalize(transformedSelect, resolverToBe);
-                }
-                parallelIteratorFactory = hasLimit ? null : new DeletingParallelIteratorFactory(connection);
-                QueryOptimizer optimizer = new QueryOptimizer(services);
-                QueryCompiler compiler = new QueryCompiler(statement, select, resolverToBe, Collections.<PColumn>emptyList(), parallelIteratorFactory, new SequenceManager(statement));
-                dataPlanToBe = compiler.compile();
-                queryPlans = Lists.newArrayList(mayHaveImmutableIndexes
-                        ? optimizer.getApplicablePlans(dataPlanToBe, statement, select, resolverToBe, Collections.<PColumn>emptyList(), parallelIteratorFactory)
-                        : optimizer.getBestPlan(dataPlanToBe, statement, select, resolverToBe, Collections.<PColumn>emptyList(), parallelIteratorFactory));
-                if (mayHaveImmutableIndexes) { // FIXME: this is ugly
-                    // Lookup the table being deleted from in the cache, as it's possible that the
-                    // optimizer updated the cache if it found indexes that were out of date.
-                    // If the index was marked as disabled, it should not be in the list
-                    // of immutable indexes.
-                    table = connection.getTable(new PTableKey(table.getTenantId(), table.getName().getString()));
-                    tableRefToBe.setTable(table);
-                    immutableIndex = getNonDisabledGlobalImmutableIndexes(tableRefToBe);
-                }
-            } catch (MetaDataEntityNotFoundException e) {
-                // Catch column/column family not found exception, as our meta data may
-                // be out of sync. Update the cache once and retry if we were out of sync.
-                // Otherwise throw, as we'll just get the same error next time.
-                if (retryOnce) {
-                    retryOnce = false;
-                    MetaDataMutationResult result = new MetaDataClient(connection).updateCache(schemaName, tableName);
-                    if (result.wasUpdated()) {
-                        continue;
-                    }
-                }
-                throw e;
-            }
-            break;
-        }
-        boolean isBuildingImmutable = false;
-        final boolean hasImmutableIndexes = !immutableIndex.isEmpty();
-        if (hasImmutableIndexes) {
-            for (PTable index : immutableIndex.values()){
-                if (index.getIndexState() == PIndexState.BUILDING) {
-                    isBuildingImmutable = true;
-                    break;
-                }
-            }
+        List<PTable> immutableIndexes = getNonDisabledGlobalImmutableIndexes(targetTableRef);
+        final boolean hasImmutableIndexes = !immutableIndexes.isEmpty();
+
+        boolean isSalted = table.getBucketNum() != null;
+        boolean isMultiTenant = connection.getTenantId() != null && table.isMultiTenant();
+        boolean isSharedViewIndex = table.getViewIndexId() != null;
+        int pkColumnOffset = (isSalted ? 1 : 0) + (isMultiTenant ? 1 : 0) + (isSharedViewIndex ? 1 : 0);
+        final int pkColumnCount = table.getPKColumns().size() - pkColumnOffset;
+        int selectColumnCount = pkColumnCount;
+        for (PTable index : immutableIndexes) {
+            selectColumnCount += index.getPKColumns().size() - pkColumnCount;
         }
-        final QueryPlan dataPlan = dataPlanToBe;
-        // tableRefs is parallel with queryPlans
-        TableRef[] tableRefs = new TableRef[hasImmutableIndexes ? immutableIndex.size() : 1];
-        if (hasImmutableIndexes) {
-            int i = 0;
-            Iterator<QueryPlan> plans = queryPlans.iterator();
-            while (plans.hasNext()) {
-                QueryPlan plan = plans.next();
-                PTable table = plan.getTableRef().getTable();
-                if (table.getType() == PTableType.INDEX) { // index plans
-                    tableRefs[i++] = plan.getTableRef();
-                    immutableIndex.remove(table.getKey());
-                } else if (!isBuildingImmutable) { // data plan
-                    /*
-                     * If we have immutable indexes that we need to maintain, don't execute the data plan
-                     * as we can save a query by piggy-backing on any of the other index queries, since the
-                     * PK columns that we need are always in each index row.
-                     */
-                    plans.remove();
+        List<PColumn> projectedColumns = Lists.newArrayListWithExpectedSize(selectColumnCount + pkColumnOffset);
+        List<AliasedNode> aliasedNodes = Lists.newArrayListWithExpectedSize(selectColumnCount);
+        for (int i = isSalted ? 1 : 0; i < pkColumnOffset; i++) {
+            PColumn column = table.getPKColumns().get(i);
+            projectedColumns.add(column);
+        }
+        for (int i = pkColumnOffset; i < table.getPKColumns().size(); i++) {
+            PColumn column = table.getPKColumns().get(i);
+            projectedColumns.add(column);
+            aliasedNodes.add(FACTORY.aliasedNode(null, FACTORY.column(null, '"' + column.getName().getString() + '"', null)));
+        }
+        // Project all non PK indexed columns so that we can do the proper index maintenance
+        for (PTable index : table.getIndexes()) {
+            IndexMaintainer maintainer = index.getIndexMaintainer(table, connection);
+            // Go through maintainer as it handles functional indexes correctly
+            for (Pair<String,String> columnInfo : maintainer.getIndexedColumnInfo()) {
+                String familyName = columnInfo.getFirst();
+                if (familyName != null) {
+                    String columnName = columnInfo.getSecond();
+                    boolean hasNoColumnFamilies = table.getColumnFamilies().isEmpty();
+                    PColumn column = hasNoColumnFamilies ? table.getColumnForColumnName(columnName) : table.getColumnFamily(familyName).getPColumnForColumnName(columnName);
+                    projectedColumns.add(column);
+                    aliasedNodes.add(FACTORY.aliasedNode(null, FACTORY.column(hasNoColumnFamilies ? null : TableName.create(null, familyName), '"' + columnName + '"', null)));
                 }
             }
-            /*
-             * If we have any immutable indexes remaining, then that means that the plan for that index got filtered out
-             * because it could not be executed. This would occur if a column in the where clause is not found in the
-             * immutable index.
-             */
-            if (!immutableIndex.isEmpty()) {
-                Collection<PTable> immutableIndexes = immutableIndex.values();
-                if (!isBuildingImmutable || hasNonPKIndexedColumns(immutableIndexes)) {
-                    throw new SQLExceptionInfo.Builder(SQLExceptionCode.INVALID_FILTER_ON_IMMUTABLE_ROWS).setSchemaName(tableRefToBe.getTable().getSchemaName().getString())
-                    .setTableName(tableRefToBe.getTable().getTableName().getString()).build().buildException();
+        }
+        select = FACTORY.select(delete.getTable(), delete.getHint(), false, aliasedNodes, delete.getWhere(),
+                Collections.<ParseNode> emptyList(), null, delete.getOrderBy(), delete.getLimit(), null,
+                delete.getBindCount(), false, false, Collections.<SelectStatement> emptyList(),
+                delete.getUdfParseNodes());
+        select = StatementNormalizer.normalize(select, resolverToBe);
+        
+        SelectStatement transformedSelect = SubqueryRewriter.transform(select, resolverToBe, connection);
+        boolean hasPreProcessing = transformedSelect != select;
+        if (transformedSelect != select) {
+            resolverToBe = FromCompiler.getResolverForQuery(transformedSelect, connection, false, delete.getTable().getName());
+            select = StatementNormalizer.normalize(transformedSelect, resolverToBe);
+        }
+        final boolean hasPreOrPostProcessing = hasPreProcessing || hasPostProcessing;
+        boolean noQueryReqd = !hasPreOrPostProcessing;
+        // No limit and no sub queries, joins, etc in where clause
+        // Can't run on same server for transactional data, as we need the row keys for the data
+        // that is being upserted for conflict detection purposes.
+        // If we have immutable indexes, we'd increase the number of bytes scanned by executing
+        // separate queries against each index, so better to drive from a single table in that case.
+        boolean runOnServer = isAutoCommit && !hasPreOrPostProcessing && !table.isTransactional() && !hasImmutableIndexes;
+        HintNode hint = delete.getHint();
+        if (runOnServer && !delete.getHint().hasHint(Hint.USE_INDEX_OVER_DATA_TABLE)) {
+            select = SelectStatement.create(select, HintNode.create(hint, Hint.USE_DATA_OVER_INDEX_TABLE));
+        }
+        
+        parallelIteratorFactoryToBe = hasPreOrPostProcessing ? null : new DeletingParallelIteratorFactory(connection);
+        QueryOptimizer optimizer = new QueryOptimizer(services);
+        QueryCompiler compiler = new QueryCompiler(statement, select, resolverToBe, Collections.<PColumn>emptyList(), parallelIteratorFactoryToBe, new SequenceManager(statement));
+        final QueryPlan dataPlan = compiler.compile();
+        // TODO: the select clause should know that there's a sub query, but doesn't seem to currently
+        queryPlans = Lists.newArrayList(!immutableIndexes.isEmpty()
+                ? optimizer.getApplicablePlans(dataPlan, statement, select, resolverToBe, Collections.<PColumn>emptyList(), parallelIteratorFactoryToBe)
+                : optimizer.getBestPlan(dataPlan, statement, select, resolverToBe, Collections.<PColumn>emptyList(), parallelIteratorFactoryToBe));
+        // Filter out any local indexes that don't contain all indexed columns.
+        // We have to do this manually because local indexes are still used
+        // when referenced columns aren't in the index, so they won't be
+        // filtered by the optimizer.
+        queryPlans = new ArrayList<>(queryPlans);
+        Iterator<QueryPlan> iterator = queryPlans.iterator();
+        while (iterator.hasNext()) {
+            QueryPlan plan = iterator.next();
+            if (plan.getTableRef().getTable().getIndexType() == IndexType.LOCAL) {
+                if (!plan.getContext().getDataColumns().isEmpty()) {
+                    iterator.remove();
                 }
-                runOnServer = false;
-            }
+            }            
         }
-        List<TableRef> buildingImmutableIndexes = Lists.newArrayListWithExpectedSize(immutableIndex.values().size());
-        for (PTable index : immutableIndex.values()) {
-            buildingImmutableIndexes.add(new TableRef(index, dataPlan.getTableRef().getTimeStamp(), dataPlan.getTableRef().getLowerBoundTimeStamp()));
+        if (queryPlans.isEmpty()) {
+            queryPlans = Collections.singletonList(dataPlan);
         }
         
-        // Make sure the first plan is targeting deletion from the data table
-        // In the case of an immutable index, we'll also delete from the index.
-        final TableRef dataTableRef = tableRefs[0] = tableRefToBe;
-        /*
-         * Create a mutationPlan for each queryPlan. One plan will be for the deletion of the rows
-         * from the data table, while the others will be for deleting rows from immutable indexes.
-         */
-        List<MutationPlan> mutationPlans = Lists.newArrayListWithExpectedSize(tableRefs.length);
-        for (int i = 0; i < tableRefs.length; i++) {
-            final TableRef tableRef = tableRefs[i];
-            final QueryPlan plan = queryPlans.get(i);
-            if (!plan.getTableRef().equals(tableRef) || !(plan instanceof BaseQueryPlan)) {
-                runOnServer = false;
-                noQueryReqd = false; // FIXME: why set this to false in this case?
-            }
-            
-            final int maxSize = services.getProps().getInt(QueryServices.MAX_MUTATION_SIZE_ATTRIB,QueryServicesOptions.DEFAULT_MAX_MUTATION_SIZE);
-            final int maxSizeBytes = services.getProps().getInt(QueryServices.MAX_MUTATION_SIZE_BYTES_ATTRIB,QueryServicesOptions.DEFAULT_MAX_MUTATION_SIZE_BYTES);
-     
-            final StatementContext context = plan.getContext();
-            // If we're doing a query for a set of rows with no where clause, then we don't need to contact the server at all.
-            // A simple check of the none existence of a where clause in the parse node is not sufficient, as the where clause
-            // may have been optimized out. Instead, we check that there's a single SkipScanFilter
-            if (noQueryReqd
-                    && (!context.getScan().hasFilter()
-                        || context.getScan().getFilter() instanceof SkipScanFilter)
-                    && context.getScanRanges().isPointLookup()) {
+        runOnServer &= queryPlans.get(0).getTableRef().getTable().getType() != PTableType.INDEX;
+        
+        // We need to have all indexed columns available in all immutable indexes in order
+        // to generate the delete markers from the query. We also cannot have any filters
+        // except for our SkipScanFilter for point lookups.
+        // A simple check of the non existence of a where clause in the parse node is not sufficient, as the where clause
+        // may have been optimized out. Instead, we check that there's a single SkipScanFilter
+        // If we can generate a plan for every index, that means all the required columns are available in every index,
+        // hence we can drive the delete from any of the plans.
+        noQueryReqd &= queryPlans.size() == 1 + immutableIndexes.size();
+        int queryPlanIndex = 0;
+        while (noQueryReqd && queryPlanIndex < queryPlans.size()) {
+            QueryPlan plan = queryPlans.get(queryPlanIndex++);
+            StatementContext context = plan.getContext();
+            noQueryReqd &= (!context.getScan().hasFilter()
+                    || context.getScan().getFilter() instanceof SkipScanFilter)
+                && context.getScanRanges().isPointLookup();
+        }
+
+        final int maxSize = services.getProps().getInt(QueryServices.MAX_MUTATION_SIZE_ATTRIB,QueryServicesOptions.DEFAULT_MAX_MUTATION_SIZE);
+        final int maxSizeBytes = services.getProps().getInt(QueryServices.MAX_MUTATION_SIZE_BYTES_ATTRIB,QueryServicesOptions.DEFAULT_MAX_MUTATION_SIZE_BYTES);
+ 
+        // If we're doing a query for a set of rows with no where clause, then we don't need to contact the server at all.
+        if (noQueryReqd) {
+            // Create a mutationPlan for each queryPlan. One plan will be for the deletion of the rows
+            // from the data table, while the others will be for deleting rows from immutable indexes.
+            List<MutationPlan> mutationPlans = Lists.newArrayListWithExpectedSize(queryPlans.size());
+            for (final QueryPlan plan : queryPlans) {
+                final StatementContext context = plan.getContext();
                 mutationPlans.add(new MutationPlan() {
     
                     @Override
@@ -561,7 +564,7 @@ public class DeleteCompiler {
                         while (iterator.hasNext()) {
                             mutation.put(new ImmutableBytesPtr(iterator.next().getLowerRange()), new RowMutationState(PRow.DELETE_MARKER, statement.getConnection().getStatementExecutionCounter(), NULL_ROWTIMESTAMP_INFO, null));
                         }
-                        return new MutationState(tableRef, mutation, 0, maxSize, maxSizeBytes, connection);
+                        return new MutationState(context.getCurrentTable(), mutation, 0, maxSize, maxSizeBytes, connection);
                     }
     
                     @Override
@@ -576,7 +579,7 @@ public class DeleteCompiler {
 
                     @Override
                     public TableRef getTargetRef() {
-                        return dataTableRef;
+                        return dataPlan.getTableRef();
                     }
 
                     @Override
@@ -605,202 +608,230 @@ public class DeleteCompiler {
                         return 0l;
                     }
                 });
-            } else if (runOnServer) {
-                // TODO: better abstraction
-                Scan scan = context.getScan();
-                scan.setAttribute(BaseScannerRegionObserver.DELETE_AGG, QueryConstants.TRUE);
+            }
+            return new MultiDeleteMutationPlan(mutationPlans);
+        } else if (runOnServer) {
+            // TODO: better abstraction
+            final StatementContext context = dataPlan.getContext();
+            Scan scan = context.getScan();
+            scan.setAttribute(BaseScannerRegionObserver.DELETE_AGG, QueryConstants.TRUE);
+
+            // Build an ungrouped aggregate query: select COUNT(*) from <table> where <where>
+            // The coprocessor will delete each row returned from the scan
+            // Ignoring ORDER BY, since with auto commit on and no limit makes no difference
+            SelectStatement aggSelect = SelectStatement.create(SelectStatement.COUNT_ONE, delete.getHint());
+            RowProjector projectorToBe = ProjectionCompiler.compile(context, aggSelect, GroupBy.EMPTY_GROUP_BY);
+            context.getAggregationManager().compile(context, GroupBy.EMPTY_GROUP_BY);
+            if (dataPlan.getProjector().projectEveryRow()) {
+                projectorToBe = new RowProjector(projectorToBe,true);
+            }
+            final RowProjector projector = projectorToBe;
+            final QueryPlan aggPlan = new AggregatePlan(context, select, dataPlan.getTableRef(), projector, null, null,
+                    OrderBy.EMPTY_ORDER_BY, null, GroupBy.EMPTY_GROUP_BY, null);
+            return new MutationPlan() {
+                        @Override
+                        public ParameterMetaData getParameterMetaData() {
+                            return context.getBindManager().getParameterMetaData();
+                        }
+        
+                        @Override
+                        public StatementContext getContext() {
+                            return context;
+                        }
+        
+                        @Override
+                        public TableRef getTargetRef() {
+                            return dataPlan.getTableRef();
+                        }
     
-                // Build an ungrouped aggregate query: select COUNT(*) from <table> where <where>
-                // The coprocessor will delete each row returned from the scan
-                // Ignoring ORDER BY, since with auto commit on and no limit makes no difference
-                SelectStatement aggSelect = SelectStatement.create(SelectStatement.COUNT_ONE, delete.getHint());
-                RowProjector projectorToBe = ProjectionCompiler.compile(context, aggSelect, GroupBy.EMPTY_GROUP_BY);
-                context.getAggregationManager().compile(context, GroupBy.EMPTY_GROUP_BY);
-                if (plan.getProjector().projectEveryRow()) {
-                    projectorToBe = new RowProjector(projectorToBe,true);
-                }
-                final RowProjector projector = projectorToBe;
-                final QueryPlan aggPlan = new AggregatePlan(context, select, tableRef, projector, null, null,
-                        OrderBy.EMPTY_ORDER_BY, null, GroupBy.EMPTY_GROUP_BY, null);
-                mutationPlans.add(new MutationPlan() {
-                    @Override
-                    public ParameterMetaData getParameterMetaData() {
-                        return context.getBindManager().getParameterMetaData();
-                    }
+                        @Override
+                        public Set<TableRef> getSourceRefs() {
+                            return dataPlan.getSourceRefs();
+                        }
     
-                    @Override
-                    public StatementContext getContext() {
-                        return context;
-                    }
+                		@Override
+                		public Operation getOperation() {
+                			return operation;
+                		}
     
-                    @Override
-                    public TableRef getTargetRef() {
-                        return dataTableRef;
-                    }
-
-                    @Override
-                    public Set<TableRef> getSourceRefs() {
-                        return dataPlan.getSourceRefs();
-                    }
-
-            		@Override
-            		public Operation getOperation() {
-            			return operation;
-            		}
-
-                    @Override
-                    public MutationState execute() throws SQLException {
-                        // TODO: share this block of code with UPSERT SELECT
-                        ImmutableBytesWritable ptr = context.getTempPtr();
-                        PTable table = tableRef.getTable();
-                        table.getIndexMaintainers(ptr, context.getConnection());
-                        byte[] txState = table.isTransactional() ? connection.getMutationState().encodeTransaction() : ByteUtil.EMPTY_BYTE_ARRAY;
-                        ServerCache cache = null;
-                        try {
-                            if (ptr.getLength() > 0) {
-                                byte[] uuidValue = ServerCacheClient.generateId();
-                                context.getScan().setAttribute(PhoenixIndexCodec.INDEX_UUID, uuidValue);
-                                context.getScan().setAttribute(PhoenixIndexCodec.INDEX_PROTO_MD, ptr.get());
-                                context.getScan().setAttribute(BaseScannerRegionObserver.TX_STATE, txState);
-                            }
-                            ResultIterator iterator = aggPlan.iterator();
+                        @Override
+                        public MutationState execute() throws SQLException {
+                            // TODO: share this block of code with UPSERT SELECT
+                            ImmutableBytesWritable ptr = context.getTempPtr();
+                            PTable table = dataPlan.getTableRef().getTable();
+                            table.getIndexMaintainers(ptr, context.getConnection());
+                            byte[] txState = table.isTransactional() ? connection.getMutationState().encodeTransaction() : ByteUtil.EMPTY_BYTE_ARRAY;
+                            ServerCache cache = null;
                             try {
-                                Tuple row = iterator.next();
-                                final long mutationCount = (Long)projector.getColumnProjector(0).getValue(row, PLong.INSTANCE, ptr);
-                                return new MutationState(maxSize, maxSizeBytes, connection) {
-                                    @Override
-                                    public long getUpdateCount() {
-                                        return mutationCount;
-                                    }
-                                };
+                                if (ptr.getLength() > 0) {
+                                    byte[] uuidValue = ServerCacheClient.generateId();
+                                    context.getScan().setAttribute(PhoenixIndexCodec.INDEX_UUID, uuidValue);
+                                    context.getScan().setAttribute(PhoenixIndexCodec.INDEX_PROTO_MD, ptr.get());
+                                    context.getScan().setAttribute(BaseScannerRegionObserver.TX_STATE, txState);
+                                }
+                                ResultIterator iterator = aggPlan.iterator();
+                                try {
+                                    Tuple row = iterator.next();
+                                    final long mutationCount = (Long)projector.getColumnProjector(0).getValue(row, PLong.INSTANCE, ptr);
+                                    return new MutationState(maxSize, maxSizeBytes, connection) {
+                                        @Override
+                                        public long getUpdateCount() {
+                                            return mutationCount;
+                                        }
+                                    };
+                                } finally {
+                                    iterator.close();
+                                }
                             } finally {
-                                iterator.close();
-                            }
-                        } finally {
-                            if (cache != null) {
-                                cache.close();
+                                if (cache != null) {
+                                    cache.close();
+                                }
                             }
                         }
-                    }
+        
+                        @Override
+                        public ExplainPlan getExplainPlan() throws SQLException {
+                            List<String> queryPlanSteps =  aggPlan.getExplainPlan().getPlanSteps();
+                            List<String> planSteps = Lists.newArrayListWithExpectedSize(queryPlanSteps.size()+1);
+                            planSteps.add("DELETE ROWS");
+                            planSteps.addAll(queryPlanSteps);
+                            return new ExplainPlan(planSteps);
+                        }
     
-                    @Override
-                    public ExplainPlan getExplainPlan() throws SQLException {
-                        List<String> queryPlanSteps =  aggPlan.getExplainPlan().getPlanSteps();
-                        List<String> planSteps = Lists.newArrayListWithExpectedSize(queryPlanSteps.size()+1);
-                        planSteps.add("DELETE ROWS");
-                        planSteps.addAll(queryPlanSteps);
-                        return new ExplainPlan(planSteps);
-                    }
-
-                    @Override
-                    public Long getEstimatedRowsToScan() throws SQLException {
-                        return aggPlan.getEstimatedRowsToScan();
-                    }
-
-                    @Override
-                    public Long getEstimatedBytesToScan() throws SQLException {
-                        return aggPlan.getEstimatedBytesToScan();
-                    }
-
-                    @Override
-                    public Long getEstimateInfoTimestamp() throws SQLException {
-                        return aggPlan.getEstimateInfoTimestamp();
-                    }
-                });
-            } else {
-                List<TableRef> immutableIndexRefsToBe = Lists.newArrayListWithExpectedSize(dataPlan.getTableRef().getTable().getIndexes().size());
-                if (!buildingImmutableIndexes.isEmpty()) {
-                    immutableIndexRefsToBe = buildingImmutableIndexes;
-                } else if (hasImmutableIndexes && !plan.getTableRef().equals(tableRef)) {
-                    immutableIndexRefsToBe = Collections.singletonList(plan.getTableRef());
-                }
-                final List<TableRef> immutableIndexRefs = immutableIndexRefsToBe;
-                final DeletingParallelIteratorFactory parallelIteratorFactory2 = parallelIteratorFactory;
-                mutationPlans.add( new MutationPlan() {
-                    @Override
-                    public ParameterMetaData getParameterMetaData() {
-                        return context.getBindManager().getParameterMetaData();
-                    }
+                        @Override
+                        public Long getEstimatedRowsToScan() throws SQLException {
+                            return aggPlan.getEstimatedRowsToScan();
+                        }
     
-                    @Override
-                    public StatementContext getContext() {
-                        return context;
-                    }
+                        @Override
+                        public Long getEstimatedBytesToScan() throws SQLException {
+                            return aggPlan.getEstimatedBytesToScan();
+                        }
     
+                        @Override
+                        public Long getEstimateInfoTimestamp() throws SQLException {
+                            return aggPlan.getEstimateInfoTimestamp();
+                        }
+                    };
+        } else {
+            final DeletingParallelIteratorFactory parallelIteratorFactory = parallelIteratorFactoryToBe;
+            List<PColumn> adjustedProjectedColumns = Lists.newArrayListWithExpectedSize(projectedColumns.size());
+            final int offset = table.getBucketNum() == null ? 0 : 1;
+            for (int i = 0; i < projectedColumns.size(); i++) {
+                final int position = i;
+                adjustedProjectedColumns.add(new DelegateColumn(projectedColumns.get(i)) {
                     @Override
-                    public TableRef getTargetRef() {
-                        return dataTableRef;
-                    }
-
-                    @Override
-                    public Set<TableRef> getSourceRefs() {
-                        return dataPlan.getSourceRefs();
+                    public int getPosition() {
+                        return position + offset;
                     }
+                });
+            }
+            PTable projectedTable = PTableImpl.makePTable(table, PTableType.PROJECTED, adjustedProjectedColumns);
+            final TableRef projectedTableRef = new TableRef(projectedTable, targetTableRef.getLowerBoundTimeStamp(), targetTableRef.getTimeStamp());
+
+            QueryPlan bestPlanToBe = dataPlan;
+            for (QueryPlan plan : queryPlans) {
+                PTable planTable = plan.getTableRef().getTable();
+                if (planTable.getIndexState() != PIndexState.BUILDING) {
+                    bestPlanToBe = plan;
+                    break;
+                }
+            }
+            final QueryPlan bestPlan = bestPlanToBe;
+            final List<TableRef>otherTableRefs = Lists.newArrayListWithExpectedSize(immutableIndexes.size());
+            for (PTable index : immutableIndexes) {
+                if (!bestPlan.getTableRef().getTable().equals(index)) {
+                    otherTableRefs.add(new TableRef(index, targetTableRef.getLowerBoundTimeStamp(), targetTableRef.getTimeStamp()));
+                }
+            }
+            
+            if (!bestPlan.getTableRef().getTable().equals(targetTableRef.getTable())) {
+                otherTableRefs.add(projectedTableRef);
+            }
+            final StatementContext context = bestPlan.getContext();
+            return new MutationPlan() {
+                @Override
+                public ParameterMetaData getParameterMetaData() {
+                    return context.getBindManager().getParameterMetaData();
+                }
 
-            		@Override
-            		public Operation getOperation() {
-            			return operation;
-            		}
+                @Override
+                public StatementContext getContext() {
+                    return context;
+                }
 
-                    @Override
-                    public MutationState execute() throws SQLException {
-                        ResultIterator iterator = plan.iterator();
-                        try {
-                            if (!hasLimit) {
-                                Tuple tuple;
-                                long totalRowCount = 0;
-                                if (parallelIteratorFactory2 != null) {
-                                    parallelIteratorFactory2.setRowProjector(plan.getProjector());
-                                    parallelIteratorFactory2.setTargetTableRef(tableRef);
-                                    parallelIteratorFactory2.setSourceTableRef(plan.getTableRef());
-                                    parallelIteratorFactory2.setIndexTargetTableRefs(immutableIndexRefs);
-                                }
-                                while ((tuple=iterator.next()) != null) {// Runs query
-                                    Cell kv = tuple.getValue(0);
-                                    totalRowCount += PLong.INSTANCE.getCodec().decodeLong(kv.getValueArray(), kv.getValueOffset(), SortOrder.getDefault());
-                                }
-                                // Return total number of rows that have been delete. In the case of auto commit being off
-                                // the mutations will all be in the mutation state of the current connection.
-                                MutationState state = new MutationState(maxSize, maxSizeBytes, connection, totalRowCount);
+                @Override
+                public TableRef getTargetRef() {
+                    return targetTableRef;
+                }
 
-                                // set the read metrics accumulated in the parent context so that it can be published when the mutations are committed.
-                                state.setReadMetricQueue(plan.getContext().getReadMetricsQueue());
+                @Override
+                public Set<TableRef> getSourceRefs() {
+                    return dataPlan.getSourceRefs();
+                }
 
-                                return state;
-                            } else {
-                                return deleteRows(plan.getContext(), tableRef, immutableIndexRefs, iterator, plan.getProjector(), plan.getTableRef());
+        		@Override
+        		public Operation getOperation() {
+        			return operation;
+        		}
+
+                @Override
+                public MutationState execute() throws SQLException {
+                    ResultIterator iterator = bestPlan.iterator();
+                    try {
+                        if (!hasPreOrPostProcessing) {
+                            Tuple tuple;
+                            long totalRowCount = 0;
+                            if (parallelIteratorFactory != null) {
+                                parallelIteratorFactory.setQueryPlan(bestPlan);
+                                parallelIteratorFactory.setOtherTableRefs(otherTableRefs);
+                                parallelIteratorFactory.setProjectedTableRef(projectedTableRef);
                             }
-                        } finally {
-                            iterator.close();
+                            while ((tuple=iterator.next()) != null) {// Runs query
+                                Cell kv = tuple.getValue(0);
+                                totalRowCount += PLong.INSTANCE.getCodec().decodeLong(kv.getValueArray(), kv.getValueOffset(), SortOrder.getDefault());
+                            }
+                            // Return total number of rows that have been deleted from the table. In the case of auto commit being off
+                            // the mutations will all be in the mutation state of the current connection. We need to divide by the
+                            // total number of tables we updated as otherwise the client will get an unexpected result
+                            MutationState state = new MutationState(maxSize, maxSizeBytes, connection, totalRowCount / ((bestPlan.getTableRef().getTable().getIndexType() == IndexType.LOCAL && !otherTableRefs.isEmpty() ? 0 : 1) + otherTableRefs.size()));
+
+                            // set the read metrics accumulated in the parent context so that it can be published when the mutations are committed.
+                            state.setReadMetricQueue(context.getReadMetricsQueue());
+
+                            return state;
+                        } else {
+                            return deleteRows(context, iterator, bestPlan, projectedTableRef, otherTableRefs);
                         }
+                    } finally {
+                        iterator.close();
                     }
-    
-                    @Override
-                    public ExplainPlan getExplainPlan() throws SQLException {
-                        List<String> queryPlanSteps =  plan.getExplainPlan().getPlanSteps();
-                        List<String> planSteps = Lists.newArrayListWithExpectedSize(queryPlanSteps.size()+1);
-                        planSteps.add("DELETE ROWS");
-                        planSteps.addAll(queryPlanSteps);
-                        return new ExplainPlan(planSteps);
-                    }
+                }
 
-                    @Override
-                    public Long getEstimatedRowsToScan() throws SQLException {
-                        return plan.getEstimatedRowsToScan();
-                    }
+                @Override
+                public ExplainPlan getExplainPlan() throws SQLException {
+                    List<String> queryPlanSteps =  bestPlan.getExplainPlan().getPlanSteps();
+                    List<String> planSteps = Lists.newArrayListWithExpectedSize(queryPlanSteps.size()+1);
+                    planSteps.add("DELETE ROWS");
+                    planSteps.addAll(queryPlanSteps);
+                    return new ExplainPlan(planSteps);
+                }
 
-                    @Override
-                    public Long getEstimatedBytesToScan() throws SQLException {
-                        return plan.getEstimatedBytesToScan();
-                    }
+                @Override
+                public Long getEstimatedRowsToScan() throws SQLException {
+                    return bestPlan.getEstimatedRowsToScan();
+                }
 
-                    @Override
-                    public Long getEstimateInfoTimestamp() throws SQLException {
-                        return plan.getEstimateInfoTimestamp();
-                    }
-                });
-            }
+                @Override
+                public Long getEstimatedBytesToScan() throws SQLException {
+                    return bestPlan.getEstimatedBytesToScan();
+                }
+
+                @Override
+                public Long getEstimateInfoTimestamp() throws SQLException {
+                    return bestPlan.getEstimateInfoTimestamp();
+                }
+            };
         }
-        return mutationPlans.size() == 1 ? mutationPlans.get(0) : new MultiDeleteMutationPlan(mutationPlans);
     }
 }
\ No newline at end of file


[14/20] phoenix git commit: PHOENIX-4287 Add null check for parent name

Posted by ja...@apache.org.
PHOENIX-4287 Add null check for parent name


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

Branch: refs/heads/5.x-HBase-2.0
Commit: cdec8149886581f4a3bffa4fe6c9cdf2e8c0ff75
Parents: 072ff58
Author: Samarth Jain <sa...@apache.org>
Authored: Thu Nov 2 17:52:32 2017 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Thu Nov 9 12:52:19 2017 -0800

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


http://git-wip-us.apache.org/repos/asf/phoenix/blob/cdec8149/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
index 9010a64..f6052a7 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
@@ -1246,9 +1246,10 @@ public abstract class BaseResultIterators extends ExplainTable implements Result
         }
         /*
          * For a view index, we use the property set on view. For indexes on base table, whether
-         * global or local, we use the property set on the base table.
+         * global or local, we use the property set on the base table. Null check needed when
+         * dropping local indexes.
          */
-        if (table.getType() == PTableType.INDEX) {
+        if (table.getType() == PTableType.INDEX && table.getParentName() != null) {
             PhoenixConnection conn = context.getConnection();
             String parentTableName = table.getParentName().getString();
             try {


[09/20] phoenix git commit: PHOENIX-4287 Addendum to correctly set useStatsForParallelization property

Posted by ja...@apache.org.
PHOENIX-4287 Addendum to correctly set useStatsForParallelization property


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

Branch: refs/heads/5.x-HBase-2.0
Commit: fc327e74b280fd5e5c744b7149ad349d501ebadf
Parents: 2a67137
Author: Samarth Jain <sa...@apache.org>
Authored: Wed Nov 1 21:13:40 2017 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Thu Nov 9 12:44:10 2017 -0800

----------------------------------------------------------------------
 .../end2end/ExplainPlanWithStatsEnabledIT.java  | 87 +++++++++++++++++---
 .../coprocessor/MetaDataEndpointImpl.java       |  2 +-
 .../phoenix/iterate/BaseResultIterators.java    |  9 +-
 .../apache/phoenix/schema/DelegateTable.java    |  2 +-
 .../apache/phoenix/schema/MetaDataClient.java   | 26 +++---
 .../java/org/apache/phoenix/schema/PTable.java  |  2 +-
 .../org/apache/phoenix/schema/PTableImpl.java   | 22 ++---
 7 files changed, 110 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/fc327e74/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
index 25d4194..b5e4588 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
@@ -17,7 +17,6 @@
  */
 package org.apache.phoenix.end2end;
 
-import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_USE_STATS_FOR_PARALLELIZATION;
 import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -34,6 +33,7 @@ import java.util.List;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixResultSet;
 import org.apache.phoenix.query.BaseTest;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.TableNotFoundException;
@@ -352,7 +352,7 @@ public class ExplainPlanWithStatsEnabledIT extends ParallelStatsEnabledIT {
     }
 
     @Test
-    public void testSettingUseStatsForQueryPlanProperty() throws Exception {
+    public void testSettingUseStatsForParallelizationProperty() throws Exception {
         try (Connection conn = DriverManager.getConnection(getUrl())) {
             String table = generateUniqueName();
             String ddl =
@@ -360,20 +360,31 @@ public class ExplainPlanWithStatsEnabledIT extends ParallelStatsEnabledIT {
                             + " (PK1 INTEGER NOT NULL PRIMARY KEY, KV1 VARCHAR) USE_STATS_FOR_PARALLELIZATION = false";
             conn.createStatement().execute(ddl);
             assertUseStatsForQueryFlag(table, conn.unwrap(PhoenixConnection.class), false);
+
             ddl = "ALTER TABLE " + table + " SET USE_STATS_FOR_PARALLELIZATION = true";
             conn.createStatement().execute(ddl);
             assertUseStatsForQueryFlag(table, conn.unwrap(PhoenixConnection.class), true);
+
+            table = generateUniqueName();
+            ddl =
+                    "CREATE TABLE " + table
+                            + " (PK1 INTEGER NOT NULL PRIMARY KEY, KV1 VARCHAR) USE_STATS_FOR_PARALLELIZATION = false";
+            conn.createStatement().execute(ddl);
+            assertUseStatsForQueryFlag(table, conn.unwrap(PhoenixConnection.class), false);
+
             table = generateUniqueName();
             ddl = "CREATE TABLE " + table + " (PK1 INTEGER NOT NULL PRIMARY KEY, KV1 VARCHAR)";
             conn.createStatement().execute(ddl);
-            assertUseStatsForQueryFlag(table, conn.unwrap(PhoenixConnection.class),
-                DEFAULT_USE_STATS_FOR_PARALLELIZATION);
+
+            // because we didn't set the property, PTable.useStatsForParallelization() should return
+            // null
+            assertUseStatsForQueryFlag(table, conn.unwrap(PhoenixConnection.class), null);
         }
     }
 
     private static void assertUseStatsForQueryFlag(String tableName, PhoenixConnection conn,
-            boolean flag) throws TableNotFoundException, SQLException {
-        assertEquals(flag,
+            Boolean expected) throws TableNotFoundException, SQLException {
+        assertEquals(expected,
             conn.unwrap(PhoenixConnection.class).getMetaDataCache()
                     .getTableRef(new PTableKey(null, tableName)).getTable()
                     .useStatsForParallelization());
@@ -383,7 +394,12 @@ public class ExplainPlanWithStatsEnabledIT extends ParallelStatsEnabledIT {
         stmt.setString(1, tableName);
         ResultSet rs = stmt.executeQuery();
         rs.next();
-        assertEquals(flag, rs.getBoolean(1));
+        boolean b = rs.getBoolean(1);
+        if (expected == null) {
+            assertTrue(rs.wasNull());
+        } else {
+            assertEquals(expected, b);
+        }
     }
 
     @Test
@@ -510,8 +526,7 @@ public class ExplainPlanWithStatsEnabledIT extends ParallelStatsEnabledIT {
             int guidePostWidth = 20;
             String ddl =
                     "CREATE TABLE " + tableName + " (k INTEGER PRIMARY KEY, a bigint, b bigint)"
-                            + " GUIDE_POSTS_WIDTH=" + guidePostWidth
-                            + ", USE_STATS_FOR_PARALLELIZATION=false";
+                            + " GUIDE_POSTS_WIDTH=" + guidePostWidth;
             byte[][] splits =
                     new byte[][] { Bytes.toBytes(102), Bytes.toBytes(105), Bytes.toBytes(108) };
             BaseTest.createTestTable(getUrl(), ddl, splits, null);
@@ -531,18 +546,70 @@ public class ExplainPlanWithStatsEnabledIT extends ParallelStatsEnabledIT {
         List<Object> binds = Lists.newArrayList();
         try (Connection conn = DriverManager.getConnection(getUrl())) {
             String sql = "SELECT COUNT(*) " + " FROM " + tableName;
+            // We don't have the use stats for parallelization property
+            // set on the table. In this case, we end up defaulting to the
+            // value set in config which is true.
             ResultSet rs = conn.createStatement().executeQuery(sql);
+            // stats are being used for parallelization. So number of scans is higher.
+            assertEquals(14, rs.unwrap(PhoenixResultSet.class).getStatement().getQueryPlan()
+                    .getScans().get(0).size());
             assertTrue(rs.next());
             assertEquals(10, rs.getInt(1));
             Estimate info = getByteRowEstimates(conn, sql, binds);
             assertEquals((Long) 10l, info.getEstimatedRows());
             assertTrue(info.getEstimateInfoTs() > 0);
+            
+            // Now, let's disable USE_STATS_FOR_PARALLELIZATION on the table
+            conn.createStatement().execute("ALTER TABLE " + tableName + " SET USE_STATS_FOR_PARALLELIZATION = " + false);
+            rs = conn.createStatement().executeQuery(sql);
+            // stats are not being used for parallelization. So number of scans is lower.
+            assertEquals(4, rs.unwrap(PhoenixResultSet.class).getStatement().getQueryPlan()
+                    .getScans().get(0).size());
+            assertTrue(rs.next());
+            assertEquals(10, rs.getInt(1));
+            info = getByteRowEstimates(conn, sql, binds);
+            assertEquals((Long) 10l, info.getEstimatedRows());
+            assertTrue(info.getEstimateInfoTs() > 0);
+            
+            // assert that the aggregate query on view also works correctly
+            String viewName = "V_" + generateUniqueName();
+            conn.createStatement()
+                    .execute("CREATE VIEW " + viewName + " AS SELECT * FROM " + tableName + " USE_STATS_FOR_PARALLELIZATION = false");
+            sql = "SELECT COUNT(*) FROM " + viewName;
+            rs = conn.createStatement().executeQuery(sql);
+            // stats are not being used for parallelization. So number of scans is lower.
+            assertEquals(4, rs.unwrap(PhoenixResultSet.class).getStatement().getQueryPlan()
+                    .getScans().get(0).size());
+            assertTrue(rs.next());
+            assertEquals(10, rs.getInt(1));
+            info = getByteRowEstimates(conn, sql, binds);
+            assertEquals((Long) 10l, info.getEstimatedRows());
+            assertTrue(info.getEstimateInfoTs() > 0);
 
             // Now let's make sure that when using stats for parallelization, our estimates
-            // and query results stay the same
+            // and query results stay the same for view and base table
             conn.createStatement().execute(
                 "ALTER TABLE " + tableName + " SET USE_STATS_FOR_PARALLELIZATION=true");
+            sql = "SELECT COUNT(*) FROM " + tableName;
+            // query the table
+            rs = conn.createStatement().executeQuery(sql);
+            // stats are being used for parallelization. So number of scans is higher.
+            assertEquals(14, rs.unwrap(PhoenixResultSet.class).getStatement().getQueryPlan()
+                .getScans().get(0).size());
+            assertTrue(rs.next());
+            assertEquals(10, rs.getInt(1));
+            info = getByteRowEstimates(conn, sql, binds);
+            assertEquals((Long) 10l, info.getEstimatedRows());
+            assertTrue(info.getEstimateInfoTs() > 0);
+
+            conn.createStatement().execute(
+                "ALTER TABLE " + viewName + " SET USE_STATS_FOR_PARALLELIZATION=true");
+            sql = "SELECT COUNT(*) FROM " + viewName;
+            // query the view
             rs = conn.createStatement().executeQuery(sql);
+            // stats are not being used for parallelization. So number of scans is higher.
+            assertEquals(14, rs.unwrap(PhoenixResultSet.class).getStatement().getQueryPlan()
+                .getScans().get(0).size());
             assertTrue(rs.next());
             assertEquals(10, rs.getInt(1));
             info = getByteRowEstimates(conn, sql, binds);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fc327e74/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 c2124d0..5dbf765 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
@@ -960,7 +960,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 .fromSerializedValue((byte)PTinyint.INSTANCE.toObject(encodingSchemeKv.getValueArray(),
                     encodingSchemeKv.getValueOffset(), encodingSchemeKv.getValueLength()));
         Cell useStatsForParallelizationKv = tableKeyValues[USE_STATS_FOR_PARALLELIZATION_INDEX];
-        boolean useStatsForParallelization = useStatsForParallelizationKv == null ? true : Boolean.TRUE.equals(PBoolean.INSTANCE.toObject(useStatsForParallelizationKv.getValueArray(), useStatsForParallelizationKv.getValueOffset(), useStatsForParallelizationKv.getValueLength()));
+        Boolean useStatsForParallelization = useStatsForParallelizationKv == null ? null : Boolean.TRUE.equals(PBoolean.INSTANCE.toObject(useStatsForParallelizationKv.getValueArray(), useStatsForParallelizationKv.getValueOffset(), useStatsForParallelizationKv.getValueLength()));
         
         List<PColumn> columns = Lists.newArrayListWithExpectedSize(columnCount);
         List<PTable> indexes = Lists.newArrayList();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fc327e74/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
index dce8de1..72080be 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
@@ -35,7 +35,6 @@ import java.io.ByteArrayInputStream;
 import java.io.DataInput;
 import java.io.DataInputStream;
 import java.io.EOFException;
-import java.io.IOException;
 import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.BitSet;
@@ -152,7 +151,7 @@ public abstract class BaseResultIterators extends ExplainTable implements Result
     private Long estimateInfoTimestamp;
     private boolean hasGuidePosts;
     private Scan scan;
-    private boolean useStatsForParallelization;
+    private final boolean useStatsForParallelization;
     protected Map<ImmutableBytesPtr,ServerCache> caches;
     
     static final Function<HRegionLocation, KeyRange> TO_KEY_RANGE = new Function<HRegionLocation, KeyRange>() {
@@ -492,7 +491,11 @@ public abstract class BaseResultIterators extends ExplainTable implements Result
         scanId = new UUID(ThreadLocalRandom.current().nextLong(), ThreadLocalRandom.current().nextLong()).toString();
         
         initializeScan(plan, perScanLimit, offset, scan);
-        this.useStatsForParallelization = table.useStatsForParallelization();
+        this.useStatsForParallelization =
+                table.useStatsForParallelization() == null
+                        ? context.getConnection().getQueryServices().getConfiguration().getBoolean(
+                            USE_STATS_FOR_PARALLELIZATION, DEFAULT_USE_STATS_FOR_PARALLELIZATION)
+                        : table.useStatsForParallelization();
         this.scans = getParallelScans();
         List<KeyRange> splitRanges = Lists.newArrayListWithExpectedSize(scans.size() * ESTIMATED_GUIDEPOSTS_PER_REGION);
         for (List<Scan> scanList : scans) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fc327e74/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateTable.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateTable.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateTable.java
index 3da27a7..8f15c5e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateTable.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateTable.java
@@ -312,7 +312,7 @@ public class DelegateTable implements PTable {
     }
 
     @Override
-    public boolean useStatsForParallelization() {
+    public Boolean useStatsForParallelization() {
         return delegate.useStatsForParallelization();
     }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fc327e74/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 701633b..7ce2167 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
@@ -2002,15 +2002,8 @@ public class MetaDataClient {
                 }
             }
 
-            boolean useStatsForParallelization =
-                    connection.getQueryServices().getProps().getBoolean(
-                        QueryServices.USE_STATS_FOR_PARALLELIZATION,
-                        QueryServicesOptions.DEFAULT_USE_STATS_FOR_PARALLELIZATION);
             Boolean useStatsForParallelizationProp =
                     (Boolean) TableProperty.USE_STATS_FOR_PARALLELIZATION.getValue(tableProps);
-            if (useStatsForParallelizationProp != null) {
-                useStatsForParallelization = useStatsForParallelizationProp;
-            }
 
             boolean sharedTable = statement.getTableType() == PTableType.VIEW || allocateIndexId;
             if (transactional) {
@@ -2606,7 +2599,11 @@ public class MetaDataClient {
             }
             tableUpsert.setByte(26, immutableStorageScheme.getSerializedMetadataValue());
             tableUpsert.setByte(27, encodingScheme.getSerializedMetadataValue());
-            tableUpsert.setBoolean(28, useStatsForParallelization);
+            if (useStatsForParallelizationProp == null) {
+                tableUpsert.setNull(28, Types.BOOLEAN);
+            } else {
+                tableUpsert.setBoolean(28, useStatsForParallelizationProp);
+            }
             tableUpsert.execute();
 
             if (asyncCreatedDate != null) {
@@ -2711,7 +2708,7 @@ public class MetaDataClient {
                         PTable.INITIAL_SEQ_NUM, pkName == null ? null : PNameFactory.newName(pkName), saltBucketNum, columns.values(),
                         parent == null ? null : parent.getSchemaName(), parent == null ? null : parent.getTableName(), Collections.<PTable>emptyList(), isImmutableRows,
                         physicalNames, defaultFamilyName == null ? null : PNameFactory.newName(defaultFamilyName), viewStatement, Boolean.TRUE.equals(disableWAL), multiTenant, storeNulls, viewType,
-                        result.getViewIndexId(), indexType, rowKeyOrderOptimizable, transactional, updateCacheFrequency, 0L, isNamespaceMapped, autoPartitionSeq, isAppendOnlySchema, immutableStorageScheme, encodingScheme, cqCounterToBe, useStatsForParallelization);
+                        result.getViewIndexId(), indexType, rowKeyOrderOptimizable, transactional, updateCacheFrequency, 0L, isNamespaceMapped, autoPartitionSeq, isAppendOnlySchema, immutableStorageScheme, encodingScheme, cqCounterToBe, useStatsForParallelizationProp);
                 result = new MetaDataMutationResult(code, result.getMutationTime(), table, true);
                 addTableToCache(result);
                 return table;
@@ -3304,11 +3301,12 @@ public class MetaDataClient {
                     }
                 }
                 Boolean useStatsForParallelization = null;
-                if (useStatsForParallelizationProp != null) {
-                    if (useStatsForParallelizationProp.booleanValue() != table.useStatsForParallelization()) {
-                        useStatsForParallelization = useStatsForParallelizationProp;
-                        changingPhoenixTableProperty = true;
-                    }
+                if (useStatsForParallelizationProp != null
+                        && (table.useStatsForParallelization() == null
+                                || (useStatsForParallelizationProp.booleanValue() != table
+                                        .useStatsForParallelization()))) {
+                    useStatsForParallelization = useStatsForParallelizationProp;
+                    changingPhoenixTableProperty = true;
                 }
                 Boolean isTransactional = null;
                 if (isTransactionalProp != null) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fc327e74/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
index d59e785..ec931b7 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
@@ -705,7 +705,7 @@ public interface PTable extends PMetaDataEntity {
     ImmutableStorageScheme getImmutableStorageScheme();
     QualifierEncodingScheme getEncodingScheme();
     EncodedCQCounter getEncodedCQCounter();
-    boolean useStatsForParallelization();
+    Boolean useStatsForParallelization();
     
     /**
      * Class to help track encoded column qualifier counters per column family.

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fc327e74/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 23b5161..fd84c7c 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
@@ -151,7 +151,7 @@ public class PTableImpl implements PTable {
     private ImmutableStorageScheme immutableStorageScheme;
     private QualifierEncodingScheme qualifierEncodingScheme;
     private EncodedCQCounter encodedCQCounter;
-    private boolean useStatsForParallelization;
+    private Boolean useStatsForParallelization;
 
     public PTableImpl() {
         this.indexes = Collections.emptyList();
@@ -184,7 +184,7 @@ public class PTableImpl implements PTable {
         this.isNamespaceMapped = isNamespaceMapped;
     }
     
-    public PTableImpl(PName tenantId, String schemaName, String tableName, long timestamp, List<PColumnFamily> families, boolean isNamespaceMapped, ImmutableStorageScheme storageScheme, QualifierEncodingScheme encodingScheme, boolean useStatsForParallelization) { // For base table of mapped VIEW
+    public PTableImpl(PName tenantId, String schemaName, String tableName, long timestamp, List<PColumnFamily> families, boolean isNamespaceMapped, ImmutableStorageScheme storageScheme, QualifierEncodingScheme encodingScheme, Boolean useStatsForParallelization) { // For base table of mapped VIEW
         Preconditions.checkArgument(tenantId==null || tenantId.getBytes().length > 0); // tenantId should be null or not empty
         this.tenantId = tenantId;
         this.name = PNameFactory.newName(SchemaUtil.getTableName(schemaName, tableName));
@@ -214,7 +214,7 @@ public class PTableImpl implements PTable {
     // For indexes stored in shared physical tables
     public PTableImpl(PName tenantId, PName schemaName, PName tableName, long timestamp, List<PColumnFamily> families, 
             List<PColumn> columns, List<PName> physicalNames, Short viewIndexId, boolean multiTenant, boolean isNamespaceMpped, ImmutableStorageScheme storageScheme, QualifierEncodingScheme qualifierEncodingScheme, 
-            EncodedCQCounter encodedCQCounter, boolean useStatsForParallelization) throws SQLException {
+            EncodedCQCounter encodedCQCounter, Boolean useStatsForParallelization) throws SQLException {
         this.pkColumns = this.allColumns = Collections.emptyList();
         this.rowKeySchema = RowKeySchema.EMPTY_SCHEMA;
         this.indexes = Collections.emptyList();
@@ -385,7 +385,7 @@ public class PTableImpl implements PTable {
             boolean isImmutableRows, List<PName> physicalNames, PName defaultFamilyName, String viewExpression,
             boolean disableWAL, boolean multiTenant, boolean storeNulls, ViewType viewType, Short viewIndexId,
             IndexType indexType, boolean rowKeyOrderOptimizable, boolean isTransactional, long updateCacheFrequency,
-            long indexDisableTimestamp, boolean isNamespaceMapped, String autoPartitionSeqName, boolean isAppendOnlySchema, ImmutableStorageScheme storageScheme, QualifierEncodingScheme qualifierEncodingScheme, EncodedCQCounter encodedCQCounter, boolean useStatsForParallelization) throws SQLException {
+            long indexDisableTimestamp, boolean isNamespaceMapped, String autoPartitionSeqName, boolean isAppendOnlySchema, ImmutableStorageScheme storageScheme, QualifierEncodingScheme qualifierEncodingScheme, EncodedCQCounter encodedCQCounter, Boolean useStatsForParallelization) throws SQLException {
         return new PTableImpl(tenantId, schemaName, tableName, type, state, timeStamp, sequenceNumber, pkName, bucketNum, columns, dataSchemaName,
                 dataTableName, indexes, isImmutableRows, physicalNames, defaultFamilyName,
                 viewExpression, disableWAL, multiTenant, storeNulls, viewType, viewIndexId,
@@ -401,7 +401,7 @@ public class PTableImpl implements PTable {
             IndexType indexType, boolean rowKeyOrderOptimizable, boolean isTransactional, long updateCacheFrequency,
             int baseColumnCount, long indexDisableTimestamp, boolean isNamespaceMapped,
             String autoPartitionSeqName, boolean isAppendOnlySchema, ImmutableStorageScheme storageScheme,
-            QualifierEncodingScheme qualifierEncodingScheme, EncodedCQCounter encodedCQCounter, boolean useStatsForParallelization)
+            QualifierEncodingScheme qualifierEncodingScheme, EncodedCQCounter encodedCQCounter, Boolean useStatsForParallelization)
             throws SQLException {
         return new PTableImpl(tenantId, schemaName, tableName, type, state, timeStamp, sequenceNumber, pkName,
                 bucketNum, columns, dataSchemaName, dataTableName, indexes, isImmutableRows, physicalNames,
@@ -417,7 +417,7 @@ public class PTableImpl implements PTable {
             boolean storeNulls, ViewType viewType, Short viewIndexId, IndexType indexType,
             int baseColumnCount, boolean rowKeyOrderOptimizable, boolean isTransactional, long updateCacheFrequency,
             long indexDisableTimestamp, boolean isNamespaceMapped, String autoPartitionSeqName, boolean isAppendOnlySchema, ImmutableStorageScheme storageScheme, 
-            QualifierEncodingScheme qualifierEncodingScheme, EncodedCQCounter encodedCQCounter, boolean useStatsForParallelization) throws SQLException {
+            QualifierEncodingScheme qualifierEncodingScheme, EncodedCQCounter encodedCQCounter, Boolean useStatsForParallelization) throws SQLException {
         init(tenantId, schemaName, tableName, type, state, timeStamp, sequenceNumber, pkName, bucketNum, columns,
                 parentSchemaName, parentTableName, indexes, isImmutableRows, physicalNames, defaultFamilyName,
                 viewExpression, disableWAL, multiTenant, storeNulls, viewType, viewIndexId, indexType, baseColumnCount, rowKeyOrderOptimizable,
@@ -457,7 +457,7 @@ public class PTableImpl implements PTable {
             boolean multiTenant, boolean storeNulls, ViewType viewType, Short viewIndexId,
             IndexType indexType , int baseColumnCount, boolean rowKeyOrderOptimizable, boolean isTransactional, long updateCacheFrequency, long indexDisableTimestamp, 
             boolean isNamespaceMapped, String autoPartitionSeqName, boolean isAppendOnlySchema, ImmutableStorageScheme storageScheme, QualifierEncodingScheme qualifierEncodingScheme, 
-            EncodedCQCounter encodedCQCounter, boolean useStatsForParallelization) throws SQLException {
+            EncodedCQCounter encodedCQCounter, Boolean useStatsForParallelization) throws SQLException {
         Preconditions.checkNotNull(schemaName);
         Preconditions.checkArgument(tenantId==null || tenantId.getBytes().length > 0); // tenantId should be null or not empty
         int estimatedSize = SizedUtil.OBJECT_SIZE * 2 + 23 * SizedUtil.POINTER_SIZE + 4 * SizedUtil.INT_SIZE + 2 * SizedUtil.LONG_SIZE + 2 * SizedUtil.INT_OBJECT_SIZE +
@@ -1354,7 +1354,7 @@ public class PTableImpl implements PTable {
         		}
         	}
         }
-        boolean useStatsForParallelization = true;
+        Boolean useStatsForParallelization = null;
         if (table.hasUseStatsForParallelization()) {
             useStatsForParallelization = table.getUseStatsForParallelization();
         }
@@ -1466,7 +1466,9 @@ public class PTableImpl implements PTable {
       if (table.getEncodingScheme() != null) {
           builder.setEncodingScheme(ByteStringer.wrap(new byte[]{table.getEncodingScheme().getSerializedMetadataValue()}));
       }
-      builder.setUseStatsForParallelization(table.useStatsForParallelization());
+      if (table.useStatsForParallelization() != null) {
+          builder.setUseStatsForParallelization(table.useStatsForParallelization());
+      }
       return builder.build();
     }
 
@@ -1553,7 +1555,7 @@ public class PTableImpl implements PTable {
     }
     
     @Override
-    public boolean useStatsForParallelization() {
+    public Boolean useStatsForParallelization() {
         return useStatsForParallelization;
     }
 


[16/20] phoenix git commit: PHOENIX-4237 Allow sorting on (Java) collation keys for non-English locales (Shehzaad Nakhoda)

Posted by ja...@apache.org.
PHOENIX-4237 Allow sorting on (Java) collation keys for non-English locales (Shehzaad Nakhoda)


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

Branch: refs/heads/5.x-HBase-2.0
Commit: 495387681ff97ab3d227660046d187858c264597
Parents: 8b4d604
Author: James Taylor <jt...@salesforce.com>
Authored: Fri Nov 3 09:17:29 2017 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Thu Nov 9 12:52:57 2017 -0800

----------------------------------------------------------------------
 LICENSE                                         |  43 ++--
 phoenix-core/pom.xml                            |   5 +
 .../phoenix/end2end/CollationKeyFunctionIT.java | 181 ++++++++++++++
 .../phoenix/expression/ExpressionType.java      |   4 +-
 .../function/CollationKeyFunction.java          | 199 +++++++++++++++
 .../apache/phoenix/jdbc/PhoenixConnection.java  |   3 +
 .../apache/phoenix/util/VarBinaryFormatter.java |  52 ++++
 .../function/CollationKeyFunctionTest.java      | 243 +++++++++++++++++++
 phoenix-server/pom.xml                          |   1 +
 9 files changed, 713 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/49538768/LICENSE
----------------------------------------------------------------------
diff --git a/LICENSE b/LICENSE
index 08e5e10..7bd8ad1 100644
--- a/LICENSE
+++ b/LICENSE
@@ -236,23 +236,32 @@ Font Awesome fonts (http://fontawesome.io/)
 
 3-Clause BSD License:
 
-Permission is hereby granted, free of charge, to any person obtaining a copy
-of this software and associated documentation files (the "Software"), to deal
-in the Software without restriction, including without limitation the rights
-to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
-copies of the Software, and to permit persons to whom the Software is
-furnished to do so, subject to the following conditions:
-
-The above copyright notice and this permission notice shall be included in
-all copies or substantial portions of the Software.
-
-THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
-IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
-FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
-AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
-LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
-OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
-THE SOFTWARE.
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+1. Redistributions of source code must retain the above copyright
+notice, this list of conditions and the following disclaimer.
+
+2. Redistributions in binary form must reproduce the above copyright
+notice, this list of conditions and the following disclaimer in the
+documentation and/or other materials provided with the distribution.
+
+3. Neither the name of the copyright holder nor the names of its
+contributors may be used to endorse or promote products derived from
+this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
 
 ---
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/49538768/phoenix-core/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-core/pom.xml b/phoenix-core/pom.xml
index 14f6e60..d331b78 100644
--- a/phoenix-core/pom.xml
+++ b/phoenix-core/pom.xml
@@ -471,5 +471,10 @@
       <artifactId>stream</artifactId>
       <version>${stream.version}</version>
     </dependency>
+     <dependency>
+      <groupId>com.salesforce.i18n</groupId>
+      <artifactId>i18n-util</artifactId>
+      <version>1.0.1</version>
+    </dependency>
   </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/49538768/phoenix-core/src/it/java/org/apache/phoenix/end2end/CollationKeyFunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CollationKeyFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CollationKeyFunctionIT.java
new file mode 100644
index 0000000..efbab64
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CollationKeyFunctionIT.java
@@ -0,0 +1,181 @@
+/*
+ * 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.apache.phoenix.util.TestUtil.closeStmtAndConn;
+import static org.junit.Assert.assertEquals;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.text.Collator;
+
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * End2End test that tests the COLLATION_KEY in an ORDER BY clause
+ * 
+ */
+public class CollationKeyFunctionIT extends ParallelStatsDisabledIT {
+
+	private String tableName;
+	private String[] dataArray = new String[] {
+			// (0-6) chinese characters
+			"\u963f", "\u55c4", "\u963e", "\u554a", "\u4ec8", "\u3d9a", "\u9f51",
+			// (7-13) western characters, some with accent
+			"a", "b", "ä", "A", "a", "ä", "A" };
+
+	@Before
+	public void initAndPopulateTable() throws Exception {
+		Connection conn = null;
+		PreparedStatement stmt = null;
+		tableName = generateUniqueName();
+		try {
+			conn = DriverManager.getConnection(getUrl());
+			String ddl = "CREATE TABLE " + tableName + " (id INTEGER PRIMARY KEY, data VARCHAR)";
+			conn.createStatement().execute(ddl);
+
+			// insert dataArray into the table, with the index into the array as
+			// the id
+			for (int i = 0; i < dataArray.length; i++) {
+				PreparedStatement ps = conn.prepareStatement("upsert into " + tableName + " values(?, ?)");
+				ps.setInt(1, i);
+				ps.setString(2, dataArray[i]);
+				ps.executeUpdate();
+			}
+			conn.commit();
+		} finally {
+			closeStmtAndConn(stmt, conn);
+		}
+	}
+
+	@Test
+	public void testZhSort() throws Exception {
+		queryWithCollKeyDefaultArgsWithExpectedOrder("zh", 0, 6, new Integer[] { 3, 0, 1, 6, 5, 4, 2 });
+	}
+
+	@Test
+	public void testZhTwSort() throws Exception {
+		queryWithCollKeyDefaultArgsWithExpectedOrder("zh_TW", 0, 6, new Integer[] { 0, 3, 4, 1, 5, 2, 6 });
+	}
+
+	@Test
+	public void testZhTwStrokeSort() throws Exception {
+		queryWithCollKeyDefaultArgsWithExpectedOrder("zh_TW_STROKE", 0, 6, new Integer[] { 4, 2, 0, 3, 1, 6, 5 });
+	}
+
+	@Test
+	public void testZhStrokeSort() throws Exception {
+		queryWithCollKeyDefaultArgsWithExpectedOrder("zh__STROKE", 0, 6, new Integer[] { 0, 1, 3, 4, 6, 2, 5 });
+	}
+
+	@Test
+	public void testZhPinyinSort() throws Exception {
+		queryWithCollKeyDefaultArgsWithExpectedOrder("zh__PINYIN", 0, 6, new Integer[] { 0, 1, 3, 4, 6, 2, 5 });
+	}
+
+	@Test
+	public void testUpperCaseSort() throws Exception {
+		queryWithCollKeyUpperCaseWithExpectedOrder("en", 7, 13, new Integer[] { 7, 10, 11, 13, 9, 12, 8 });
+	}
+
+	@Test
+	public void testPrimaryStrengthSort() throws Exception {
+		queryWithCollKeyWithStrengthWithExpectedOrder("en", Collator.PRIMARY, false, 7, 13,
+				new Integer[] { 7, 9, 10, 11, 12, 13, 8 });
+	}
+	
+	@Test
+	public void testSecondaryStrengthSort() throws Exception {
+		queryWithCollKeyWithStrengthWithExpectedOrder("en", Collator.SECONDARY, false, 7, 13,
+				new Integer[] { 7, 10, 11, 13, 9, 12, 8 });
+	}
+
+	@Test
+	public void testTertiaryStrengthSort() throws Exception {
+		queryWithCollKeyWithStrengthWithExpectedOrder("en", Collator.TERTIARY, false, 7, 13,
+				new Integer[] { 7, 11, 10, 13, 9, 12, 8 });
+	}
+
+	@Test
+	public void testTertiaryStrengthSortDesc() throws Exception {
+		queryWithCollKeyWithStrengthWithExpectedOrder("en", Collator.TERTIARY, true, 7, 13,
+				new Integer[] { 8, 12, 9, 13, 10, 11, 7 });
+	}
+
+	
+	/**
+	 * Issue a query ordered by the collation key (with COLLATION_KEY called
+	 * with default args) of the data column according to the provided
+	 * localeString, and compare the ID and data columns to the expected order.
+	 * 
+	 * @param expectedIndexOrder
+	 *            an array of indexes into the dataArray in the order we expect.
+	 *            This is the same as the ID column
+	 * @throws SQLException
+	 */
+	private void queryWithCollKeyDefaultArgsWithExpectedOrder(String localeString, Integer beginIndex, Integer endIndex,
+			Integer[] expectedIndexOrder) throws Exception {
+		String query = String.format(
+				"SELECT id, data FROM %s WHERE ID BETWEEN %d AND %d ORDER BY COLLATION_KEY(data, '%s')", tableName,
+				beginIndex, endIndex, localeString);
+		queryWithExpectedOrder(query, expectedIndexOrder);
+	}
+
+	/**
+	 * Same as above, except the upperCase collator argument is set to true
+	 */
+	private void queryWithCollKeyUpperCaseWithExpectedOrder(String localeString, Integer beginIndex, Integer endIndex,
+			Integer[] expectedIndexOrder) throws Exception {
+		String query = String.format(
+				"SELECT id, data FROM %s WHERE ID BETWEEN %d AND %d ORDER BY COLLATION_KEY(data, '%s', true), id",
+				tableName, beginIndex, endIndex, localeString);
+		queryWithExpectedOrder(query, expectedIndexOrder);
+	}
+
+	/**
+	 * Same as above, except the collator strength is set
+	 */
+	private void queryWithCollKeyWithStrengthWithExpectedOrder(String localeString, Integer strength, boolean isDescending,
+			Integer beginIndex, Integer endIndex, Integer[] expectedIndexOrder) throws Exception {
+		String sortOrder = isDescending ? "DESC" : "";
+		
+		String query = String.format(
+				"SELECT id, data FROM %s WHERE ID BETWEEN %d AND %d ORDER BY COLLATION_KEY(data, '%s', false, %d) %s, id %s",
+				tableName, beginIndex, endIndex, localeString, strength, sortOrder, sortOrder);
+		queryWithExpectedOrder(query, expectedIndexOrder);
+	}
+
+	private void queryWithExpectedOrder(String query, Integer[] expectedIndexOrder) throws Exception {
+		Connection conn = DriverManager.getConnection(getUrl());
+		PreparedStatement ps = conn.prepareStatement(query);
+		ResultSet rs = ps.executeQuery();
+		int i = 0;
+		while (rs.next()) {
+			int expectedId = expectedIndexOrder[i];
+			assertEquals("For row " + i + ": The ID did not match the expected index", expectedId, rs.getInt(1));
+			assertEquals("For row " + i + ": The data did not match the expected entry from the data array",
+					dataArray[expectedId], rs.getString(2));
+			i++;
+		}
+		assertEquals("The result set returned a different number of rows from the data array", expectedIndexOrder.length, i);
+	}
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/49538768/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
index 4f26e87..9a53eb1 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
@@ -42,6 +42,7 @@ import org.apache.phoenix.expression.function.CeilTimestampExpression;
 import org.apache.phoenix.expression.function.CeilWeekExpression;
 import org.apache.phoenix.expression.function.CeilYearExpression;
 import org.apache.phoenix.expression.function.CoalesceFunction;
+import org.apache.phoenix.expression.function.CollationKeyFunction;
 import org.apache.phoenix.expression.function.ConvertTimezoneFunction;
 import org.apache.phoenix.expression.function.CountAggregateFunction;
 import org.apache.phoenix.expression.function.DayOfMonthFunction;
@@ -294,7 +295,8 @@ public enum ExpressionType {
     ArrayColumnExpression(SingleCellColumnExpression.class),
     FirstValuesFunction(FirstValuesFunction.class),
     LastValuesFunction(LastValuesFunction.class),
-    DistinctCountHyperLogLogAggregateFunction(DistinctCountHyperLogLogAggregateFunction.class);
+    DistinctCountHyperLogLogAggregateFunction(DistinctCountHyperLogLogAggregateFunction.class),
+    CollationKeyFunction(CollationKeyFunction.class);
 
     ExpressionType(Class<? extends Expression> clazz) {
         this.clazz = clazz;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/49538768/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CollationKeyFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CollationKeyFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CollationKeyFunction.java
new file mode 100644
index 0000000..827f70a
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/CollationKeyFunction.java
@@ -0,0 +1,199 @@
+/*
+ * 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.expression.function;
+
+import java.io.DataInput;
+import java.io.IOException;
+import java.sql.SQLException;
+import java.text.Collator;
+import java.util.List;
+import java.util.Locale;
+
+import org.apache.commons.lang.BooleanUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.parse.FunctionParseNode;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.schema.types.PBoolean;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PInteger;
+import org.apache.phoenix.schema.types.PVarbinary;
+import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.util.VarBinaryFormatter;
+
+import com.force.db.i18n.LinguisticSort;
+import com.force.i18n.LocaleUtils;
+
+/**
+ * A Phoenix Function that calculates a collation key for an input string based
+ * on a caller-provided locale and collator strength and decomposition settings.
+ * 
+ * The locale should be specified as xx_yy_variant where xx is the ISO 639-1
+ * 2-letter language code, yy is the the ISO 3166 2-letter country code. Both
+ * countryCode and variant are optional. For example, zh_TW_STROKE, zh_TW and zh
+ * are all valid locale representations. Note the language code, country code
+ * and variant are used as arguments to the constructor of java.util.Locale.
+ *
+ * This function uses the open-source i18n-util package to obtain the collators
+ * it needs from the provided locale.
+ *
+ * The LinguisticSort implementation in i18n-util encapsulates sort-related
+ * functionality for a substantive list of locales. For each locale, it provides
+ * a collator and an Oracle-specific database function that can be used to sort
+ * strings according to the natural language rules of that locale.
+ *
+ * This function uses the collator returned by LinguisticSort.getCollator to
+ * produce a collation key for its input string. A user can expect that the
+ * sorting semantics of this function for a given locale is equivalent to the
+ * sorting behaviour of an Oracle query that is constructed using the Oracle
+ * functions returned by LinguisticSort for that locale.
+ *
+ * The optional third argument to the function is a boolean that specifies
+ * whether to use the upper-case collator (case-insensitive) returned by
+ * LinguisticSort.getUpperCaseCollator.
+ *
+ * The optional fourth and fifth arguments are used to set respectively the
+ * strength and composition of the collator returned by LinguisticSort using the
+ * setStrength and setDecomposition methods of java.text.Collator.
+ * 
+ */
+@FunctionParseNode.BuiltInFunction(name = CollationKeyFunction.NAME, args = {
+		// input string
+		@FunctionParseNode.Argument(allowedTypes = { PVarchar.class }),
+		// ISO Code for Locale
+		@FunctionParseNode.Argument(allowedTypes = { PVarchar.class }, isConstant = true),
+		// whether to use special upper case collator
+		@FunctionParseNode.Argument(allowedTypes = { PBoolean.class }, defaultValue = "false", isConstant = true),
+		// collator strength
+		@FunctionParseNode.Argument(allowedTypes = { PInteger.class }, defaultValue = "null", isConstant = true),
+		// collator decomposition
+		@FunctionParseNode.Argument(allowedTypes = { PInteger.class }, defaultValue = "null", isConstant = true) })
+public class CollationKeyFunction extends ScalarFunction {
+
+	private static final Log LOG = LogFactory.getLog(CollationKeyFunction.class);
+
+	public static final String NAME = "COLLATION_KEY";
+
+	private Collator collator;
+
+	public CollationKeyFunction() {
+	}
+
+	public CollationKeyFunction(List<Expression> children) throws SQLException {
+		super(children);
+		initialize();
+	}
+
+	@Override
+	public void readFields(DataInput input) throws IOException {
+		super.readFields(input);
+		initialize();
+	}
+
+	@Override
+	public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+		Expression expression = getChildren().get(0);
+		if (!expression.evaluate(tuple, ptr)) {
+			return false;
+		}
+		String inputString = (String) PVarchar.INSTANCE.toObject(ptr, expression.getSortOrder());
+		if (LOG.isTraceEnabled()) {
+			LOG.trace("CollationKey inputString: " + inputString);
+		}
+		byte[] collationKeyByteArray = collator.getCollationKey(inputString).toByteArray();
+
+		if (LOG.isTraceEnabled()) {
+			LOG.trace("CollationKey bytes: " + VarBinaryFormatter.INSTANCE.format(collationKeyByteArray));
+		}
+
+		ptr.set(collationKeyByteArray);
+		return true;
+	}
+
+	private void initialize() {
+		String localeISOCode = getLiteralValue(1, String.class);
+		Boolean useSpecialUpperCaseCollator = getLiteralValue(2, Boolean.class);
+		Integer collatorStrength = getLiteralValue(3, Integer.class);
+		Integer collatorDecomposition = getLiteralValue(4, Integer.class);
+
+		if (LOG.isTraceEnabled()) {
+			StringBuilder logInputsMessage = new StringBuilder();
+			logInputsMessage.append("Input (literal) arguments:").append("localeISOCode: " + localeISOCode)
+					.append(", useSpecialUpperCaseCollator: " + useSpecialUpperCaseCollator)
+					.append(", collatorStrength: " + collatorStrength)
+					.append(", collatorDecomposition: " + collatorDecomposition);
+			LOG.trace(logInputsMessage);
+		}
+
+		Locale locale = LocaleUtils.get().getLocaleByIsoCode(localeISOCode);
+
+		if (LOG.isTraceEnabled()) {
+			LOG.trace(String.format("Locale: " + locale.toLanguageTag()));
+		}
+
+		LinguisticSort linguisticSort = LinguisticSort.get(locale);
+
+		collator = BooleanUtils.isTrue(useSpecialUpperCaseCollator) ? linguisticSort.getUpperCaseCollator(false)
+				: linguisticSort.getCollator();
+
+		if (collatorStrength != null) {
+			collator.setStrength(collatorStrength);
+		}
+
+		if (collatorDecomposition != null) {
+			collator.setDecomposition(collatorDecomposition);
+		}
+
+		if (LOG.isTraceEnabled()) {
+			LOG.trace(String.format("Collator: [strength: %d, decomposition: %d], Special-Upper-Case: %s",
+					collator.getStrength(), collator.getDecomposition(),
+					BooleanUtils.isTrue(useSpecialUpperCaseCollator)));
+		}
+	}
+
+	@Override
+	public PDataType getDataType() {
+		return PVarbinary.INSTANCE;
+	}
+
+	@Override
+	public String getName() {
+		return NAME;
+	}
+
+	@Override
+	public boolean isThreadSafe() {
+		// ICU4J Collators are not thread-safe unless they are frozen.
+		// TODO: Look into calling freeze() on them to be able return true here.
+		return false;
+	}
+
+	private <T> T getLiteralValue(int childIndex, Class<T> type) {
+		Expression expression = getChildren().get(childIndex);
+		if (LOG.isDebugEnabled()) {
+			LOG.debug("child: " + childIndex + ", expression: " + expression);
+		}
+		// It's safe to assume expression is a LiteralExpression since
+		// only arguments marked as isConstant = true should be handled through
+		// this method.
+		return type.cast(((LiteralExpression) expression).getValue());
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/49538768/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
index 730f754..4555190 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
@@ -105,6 +105,7 @@ import org.apache.phoenix.schema.types.PTimestamp;
 import org.apache.phoenix.schema.types.PUnsignedDate;
 import org.apache.phoenix.schema.types.PUnsignedTime;
 import org.apache.phoenix.schema.types.PUnsignedTimestamp;
+import org.apache.phoenix.schema.types.PVarbinary;
 import org.apache.phoenix.trace.util.Tracing;
 import org.apache.phoenix.transaction.PhoenixTransactionContext;
 import org.apache.phoenix.util.DateUtil;
@@ -116,6 +117,7 @@ import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.SQLCloseable;
 import org.apache.phoenix.util.SQLCloseables;
 import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.VarBinaryFormatter;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Objects;
@@ -336,6 +338,7 @@ public class PhoenixConnection implements Connection, MetaDataMutated, SQLClosea
         formatters.put(PUnsignedTimestamp.INSTANCE, timestampFormat);
         formatters.put(PDecimal.INSTANCE,
                 FunctionArgumentType.NUMERIC.getFormatter(numberPattern));
+        formatters.put(PVarbinary.INSTANCE, VarBinaryFormatter.INSTANCE);
         // We do not limit the metaData on a connection less than the global
         // one,
         // as there's not much that will be cached here.

http://git-wip-us.apache.org/repos/asf/phoenix/blob/49538768/phoenix-core/src/main/java/org/apache/phoenix/util/VarBinaryFormatter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/VarBinaryFormatter.java b/phoenix-core/src/main/java/org/apache/phoenix/util/VarBinaryFormatter.java
new file mode 100644
index 0000000..7f0d030
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/VarBinaryFormatter.java
@@ -0,0 +1,52 @@
+/*
+ * 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.util;
+
+import java.text.FieldPosition;
+import java.text.Format;
+import java.text.ParsePosition;
+
+import org.apache.commons.codec.binary.Hex;
+
+/**
+ * A formatter that formats a byte array to a hexadecimal string
+ * (with each byte converted to a 2-digit hex sequence)
+ *
+ * @author snakhoda-sfdc
+ */
+public class VarBinaryFormatter extends Format {
+
+	private static final long serialVersionUID = -7940880118392024750L;
+
+	public static final VarBinaryFormatter INSTANCE = new VarBinaryFormatter();
+
+	@Override
+	public StringBuffer format(Object obj, StringBuffer toAppendTo, FieldPosition pos) {
+		if (!(obj instanceof byte[])) {
+			throw new IllegalArgumentException("VarBinaryFormatter can only format byte arrays");
+		}
+		String hexString = Hex.encodeHexString((byte[]) obj);
+		toAppendTo.append(hexString);
+		return toAppendTo;
+	}
+
+	@Override
+	public Object parseObject(String source, ParsePosition pos) {
+		return new UnsupportedOperationException();
+	}
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/49538768/phoenix-core/src/test/java/org/apache/phoenix/expression/function/CollationKeyFunctionTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/expression/function/CollationKeyFunctionTest.java b/phoenix-core/src/test/java/org/apache/phoenix/expression/function/CollationKeyFunctionTest.java
new file mode 100644
index 0000000..f57a937
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/expression/function/CollationKeyFunctionTest.java
@@ -0,0 +1,243 @@
+/*
+ * 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.expression.function;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.fail;
+
+import java.text.Collator;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+
+import org.apache.commons.codec.binary.Hex;
+import org.apache.commons.lang.builder.ToStringBuilder;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.schema.types.PBoolean;
+import org.apache.phoenix.schema.types.PInteger;
+import org.apache.phoenix.schema.types.PVarchar;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+import com.google.common.primitives.UnsignedBytes;
+
+/**
+ * "Unit" tests for CollationKeyFunction
+ * 
+ */
+public class CollationKeyFunctionTest {
+
+	private static String[] chineseChars = new String[] { "\u963f", "\u55c4", "\u963e", "\u554a", "\u4ec8", "\u3d9a",
+			"\u9f51" };
+
+	private static Comparator<byte[]> collationKeyComparator = UnsignedBytes.lexicographicalComparator();
+
+	private static Comparator<ByteArrayAndInteger> collationKeyAndIndexComparator = new Comparator<ByteArrayAndInteger>() {
+		@Override
+		public int compare(ByteArrayAndInteger o1, ByteArrayAndInteger o2) {
+			int compareResult = collationKeyComparator.compare(o1.byteArray, o2.byteArray);
+			if (compareResult == 0) {
+				compareResult = o1.integer.compareTo(o2.integer);
+			}
+			return compareResult;
+		}
+	};
+
+	private static class ByteArrayAndInteger {
+
+		private ByteArrayAndInteger(byte[] byteArray, Integer integer) {
+			super();
+			this.byteArray = byteArray;
+			this.integer = integer;
+		}
+
+		byte[] byteArray;
+		Integer integer;
+
+		public String toString() {
+			return ToStringBuilder.reflectionToString(this);
+		}
+
+		public static ByteArrayAndInteger findFirstIntegerMatch(List<ByteArrayAndInteger> list,
+				Integer matchingInteger) {
+			for (ByteArrayAndInteger entry : list) {
+				if (entry.integer.equals(matchingInteger)) {
+					return entry;
+				}
+			}
+			return null;
+		}
+	}
+
+	@Test
+	public void testZhSort() throws Exception {
+		testSortOrderNoEquals(chineseChars, "zh", Boolean.FALSE, null, null, new Integer[] { 3, 0, 1, 6, 5, 4, 2 });
+	}
+
+	@Test
+	public void testZhTwSort() throws Exception {
+		testSortOrderNoEquals(chineseChars, "zh_TW", Boolean.FALSE, null, null, new Integer[] { 0, 3, 4, 1, 5, 2, 6 });
+	}
+
+	@Test
+	public void testZhTwStrokeSort() throws Exception {
+		testSortOrderNoEquals(chineseChars, "zh_TW_STROKE", Boolean.FALSE, null, null,
+				new Integer[] { 4, 2, 0, 3, 1, 6, 5 });
+	}
+
+	@Test
+	public void testZhStrokeSort() throws Exception {
+		testSortOrderNoEquals(chineseChars, "zh__STROKE", Boolean.FALSE, null, null,
+				new Integer[] { 0, 1, 3, 4, 6, 2, 5 });
+	}
+
+	@Test
+	public void testZhPinyinSort() throws Exception {
+		testSortOrderNoEquals(chineseChars, "zh__PINYIN", Boolean.FALSE, null, null,
+				new Integer[] { 0, 1, 3, 4, 6, 2, 5 });
+	}
+
+	@Test
+	public void testUpperCaseCollationKeyBytes() throws Exception {
+		testCollationKeysEqual(new String[] { "abcdef", "ABCDEF", "aBcDeF" }, "en", Boolean.TRUE, null, null);
+	}
+
+	@Test
+	public void testEqualCollationKeysForPrimaryStrength() throws Exception {
+		// "a", "A", "ä" are considered equivalent
+		testCollationKeysEqual(new String[] { "a", "A", "ä" }, "en", Boolean.FALSE, Collator.PRIMARY, null);
+		testSortOrderNoEquals(new String[] { "b", "a" }, "en", Boolean.FALSE, Collator.PRIMARY, null,
+				new Integer[] { 1, 0 });
+
+	}
+
+	@Test
+	public void testCollationKeyBytesForSecondaryStrength() throws Exception {
+		// "a" and "A" are considered equivalent but not "ä"
+		testCollationKeysEqual(new String[] { "a", "A" }, "en", Boolean.FALSE, Collator.SECONDARY, null);
+		testSortOrderNoEquals(new String[] { "b", "a", "ä" }, "en", Boolean.FALSE, Collator.SECONDARY, null,
+				new Integer[] { 1, 2, 0 });
+	}
+
+	@Test
+	public void testCollationKeyBytesForTertiaryStrength() throws Exception {
+		// none of these are considered equivalent
+		testSortOrderNoEquals(new String[] { "b", "a", "ä", "A" }, "en", Boolean.FALSE, Collator.TERTIARY, null,
+				new Integer[] { 1, 3, 2, 0 });
+	}
+
+	/**
+	 * Just test that changing the decomposition mode works for basic sorting.
+	 * TODO: Actually test for the accented characters and languages where this
+	 * actually matters.
+	 */
+	@Test
+	public void testCollationKeyBytesForFullDecomposition() throws Exception {
+		testCollationKeysEqual(new String[] { "a", "A" }, "en", Boolean.FALSE, null, Collator.FULL_DECOMPOSITION);
+	}
+
+	/** HELPER METHODS **/
+	private void testSortOrderNoEquals(String[] inputStrings, String locale, Boolean uppercaseCollator,
+			Integer strength, Integer decomposition, Integer[] expectedOrder) throws Exception {
+		List<ByteArrayAndInteger> sortedCollationKeysAndIndexes = calculateCollationKeys(inputStrings, locale,
+				uppercaseCollator, strength, decomposition);
+		Collections.sort(sortedCollationKeysAndIndexes, collationKeyAndIndexComparator);
+		testCollationKeysNotEqual(inputStrings, sortedCollationKeysAndIndexes);
+
+		Integer[] sortedIndexes = new Integer[sortedCollationKeysAndIndexes.size()];
+		for (int i = 0; i < sortedIndexes.length; i++) {
+			sortedIndexes[i] = sortedCollationKeysAndIndexes.get(i).integer;
+		}
+		assertArrayEquals(expectedOrder, sortedIndexes);
+	}
+
+	private List<ByteArrayAndInteger> calculateCollationKeys(String[] inputStrings, String locale,
+			Boolean upperCaseCollator, Integer strength, Integer decomposition) throws Exception {
+		List<ByteArrayAndInteger> collationKeysAndIndexes = Lists.newArrayList();
+		for (int i = 0; i < inputStrings.length; i++) {
+			byte[] thisCollationKeyBytes = callFunction(inputStrings[i], locale, upperCaseCollator, strength,
+					decomposition, SortOrder.ASC);
+			collationKeysAndIndexes.add(new ByteArrayAndInteger(thisCollationKeyBytes, i));
+		}
+		return collationKeysAndIndexes;
+	}
+
+	private void testCollationKeysEqual(String[] inputStrings, String locale, Boolean upperCaseCollator,
+			Integer strength, Integer decomposition) throws Exception {
+		List<ByteArrayAndInteger> collationKeysAndIndexes = calculateCollationKeys(inputStrings, locale,
+				upperCaseCollator, strength, decomposition);
+
+		for (int i = 0, j = 1; i < inputStrings.length && j < inputStrings.length; i++, j++) {
+			byte[] iByteArray = ByteArrayAndInteger.findFirstIntegerMatch(collationKeysAndIndexes, i).byteArray;
+			byte[] jByteArray = ByteArrayAndInteger.findFirstIntegerMatch(collationKeysAndIndexes, j).byteArray;
+			boolean isPairEqual = collationKeyComparator.compare(iByteArray, jByteArray) == 0;
+			if (!isPairEqual) {
+				fail(String.format("Collation keys for inputStrings [%s] and [%s] ([%s], [%s]) were not equal",
+						inputStrings[i], inputStrings[j], Hex.encodeHexString(iByteArray),
+						Hex.encodeHexString(jByteArray)));
+			}
+		}
+	}
+
+	private void testCollationKeysNotEqual(String[] inputStrings, List<ByteArrayAndInteger> collationKeysAndIndexes)
+			throws Exception {
+		for (int i = 0; i < inputStrings.length; i++) {
+			for (int j = i + 1; j < inputStrings.length; j++) {
+				byte[] iByteArray = ByteArrayAndInteger.findFirstIntegerMatch(collationKeysAndIndexes, i).byteArray;
+				byte[] jByteArray = ByteArrayAndInteger.findFirstIntegerMatch(collationKeysAndIndexes, j).byteArray;
+				boolean isPairEqual = collationKeyComparator.compare(iByteArray, jByteArray) == 0;
+				if (isPairEqual) {
+					fail(String.format("Collation keys for inputStrings [%s] and [%s] ([%s], [%s]) were equal",
+							inputStrings[i], inputStrings[j], Hex.encodeHexString(iByteArray),
+							Hex.encodeHexString(jByteArray)));
+				}
+			}
+		}
+	}
+
+	private static byte[] callFunction(String inputStr, String localeIsoCode, Boolean upperCaseCollator,
+			Integer strength, Integer decomposition, SortOrder sortOrder) throws Exception {
+		LiteralExpression inputStrLiteral, localeIsoCodeLiteral, upperCaseBooleanLiteral, strengthLiteral,
+				decompositionLiteral;
+		inputStrLiteral = LiteralExpression.newConstant(inputStr, PVarchar.INSTANCE, sortOrder);
+		localeIsoCodeLiteral = LiteralExpression.newConstant(localeIsoCode, PVarchar.INSTANCE, sortOrder);
+		upperCaseBooleanLiteral = LiteralExpression.newConstant(upperCaseCollator, PBoolean.INSTANCE, sortOrder);
+		strengthLiteral = LiteralExpression.newConstant(strength, PInteger.INSTANCE, sortOrder);
+		decompositionLiteral = LiteralExpression.newConstant(decomposition, PInteger.INSTANCE, sortOrder);
+		return callFunction(inputStrLiteral, localeIsoCodeLiteral, upperCaseBooleanLiteral, strengthLiteral,
+				decompositionLiteral);
+
+	}
+
+	private static byte[] callFunction(LiteralExpression inputStrLiteral, LiteralExpression localeIsoCodeLiteral,
+			LiteralExpression upperCaseBooleanLiteral, LiteralExpression strengthLiteral,
+			LiteralExpression decompositionLiteral) throws Exception {
+		List<Expression> expressions = Lists.newArrayList((Expression) inputStrLiteral,
+				(Expression) localeIsoCodeLiteral, (Expression) upperCaseBooleanLiteral, (Expression) strengthLiteral,
+				(Expression) decompositionLiteral);
+		Expression collationKeyFunction = new CollationKeyFunction(expressions);
+		ImmutableBytesWritable ptr = new ImmutableBytesWritable();
+		boolean ret = collationKeyFunction.evaluate(null, ptr);
+		byte[] result = ret
+				? (byte[]) collationKeyFunction.getDataType().toObject(ptr, collationKeyFunction.getSortOrder()) : null;
+		return result;
+	}
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/49538768/phoenix-server/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-server/pom.xml b/phoenix-server/pom.xml
index c2e2745..6c2fe42 100644
--- a/phoenix-server/pom.xml
+++ b/phoenix-server/pom.xml
@@ -134,6 +134,7 @@
                   <include>io.dropwizard.metrics:metrics-core</include>
                   <include>org.apache.thrift:libthrift</include>
                   <include>com.clearspring.analytics:stream</include>
+                  <include>com.salesforce.i18n:i18n-util</include>
                 </includes>
                   <excludes>
                     <exclude>org.apache.phoenix:phoenix-server</exclude>


[07/20] phoenix git commit: PHOENIX-4333 Test to demonstrate partial stats information for tenant views

Posted by ja...@apache.org.
PHOENIX-4333 Test to demonstrate partial stats information for tenant views


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

Branch: refs/heads/5.x-HBase-2.0
Commit: 79d80f1b4cf79a37cef72ff1c9bebd55b2946e09
Parents: 8a19ae1
Author: Samarth Jain <sa...@apache.org>
Authored: Tue Oct 31 14:14:56 2017 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Thu Nov 9 12:43:21 2017 -0800

----------------------------------------------------------------------
 .../end2end/ExplainPlanWithStatsEnabledIT.java  | 95 ++++++++++++++++++++
 1 file changed, 95 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/79d80f1b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
index 931c398..25d4194 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
@@ -31,6 +31,7 @@ import java.sql.SQLException;
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.query.BaseTest;
@@ -782,4 +783,98 @@ public class ExplainPlanWithStatsEnabledIT extends ParallelStatsEnabledIT {
             this.time += t;
         }
     }
+
+    @Test
+    public void testPartialStatsForTenantViews() throws Exception {
+        String tenant1View = generateUniqueName();
+        String tenant2View = generateUniqueName();
+        String multiTenantTable = generateUniqueName();
+        String tenantId1 = "00Dabcdetenant1";
+        String tenantId2 = "00Dabcdetenant2";
+
+        String ddl =
+                "CREATE TABLE " + multiTenantTable
+                        + " (orgId CHAR(15) NOT NULL, pk2 CHAR(3) NOT NULL, a bigint, b bigint CONSTRAINT PK PRIMARY KEY "
+                        + "(ORGID, PK2)) MULTI_TENANT=true, GUIDE_POSTS_WIDTH=20";
+        createTestTable(getUrl(), ddl, null, null);
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            // split such that some data for view2 resides on region of view1
+            try (HBaseAdmin admin =
+                    conn.unwrap(PhoenixConnection.class).getQueryServices().getAdmin()) {
+                byte[] splitKey = Bytes.toBytes("00Dabcdetenant200B");
+                admin.split(Bytes.toBytes(multiTenantTable), splitKey);
+            }
+
+            /**
+             * Insert 2 rows for tenant1 and 6 for tenant2
+             */
+            conn.createStatement().execute(
+                "upsert into " + multiTenantTable + " values ('" + tenantId1 + "','00A',1,1)");
+            conn.createStatement().execute(
+                "upsert into " + multiTenantTable + " values ('" + tenantId1 + "','00B',2,2)");
+            conn.createStatement().execute(
+                "upsert into " + multiTenantTable + " values ('" + tenantId2 + "','00A',3,3)");
+            // We split at tenant2 + 00B. So the following rows will reside in a different region
+            conn.createStatement().execute(
+                "upsert into " + multiTenantTable + " values ('" + tenantId2 + "','00B',4,4)");
+            conn.createStatement().execute(
+                "upsert into " + multiTenantTable + " values ('" + tenantId2 + "','00C',5,5)");
+            conn.createStatement().execute(
+                "upsert into " + multiTenantTable + " values ('" + tenantId2 + "','00D',6,6)");
+            conn.createStatement().execute(
+                "upsert into " + multiTenantTable + " values ('" + tenantId2 + "','00E',7,7)");
+            conn.createStatement().execute(
+                "upsert into " + multiTenantTable + " values ('" + tenantId2 + "','00F',8,8)");
+            conn.commit();
+        }
+        try (Connection conn = getTenantConnection(tenantId1)) {
+            conn.createStatement().execute(
+                "CREATE VIEW " + tenant1View + " AS SELECT * FROM " + multiTenantTable);
+        }
+        try (Connection conn = getTenantConnection(tenantId2)) {
+            conn.createStatement().execute(
+                "CREATE VIEW " + tenant2View + " AS SELECT * FROM " + multiTenantTable);
+        }
+        String sql = "";
+        List<Object> binds = Lists.newArrayList();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            /*
+             * I have seen compaction running and generating stats for the second region of
+             * tenant2View So let's disable compaction on the table, delete any stats we have
+             * collected in SYSTEM.STATS table, clear cache and run update stats to make sure our
+             * test gets a deterministic setup.
+             */
+            String disableCompaction =
+                    "ALTER TABLE " + multiTenantTable + " SET COMPACTION_ENABLED = false";
+            conn.createStatement().executeUpdate(disableCompaction);
+            String delete =
+                    "DELETE FROM SYSTEM.STATS WHERE PHYSICAL_NAME = '" + multiTenantTable + "'";
+            conn.createStatement().executeUpdate(delete);
+            conn.commit();
+            conn.unwrap(PhoenixConnection.class).getQueryServices().clearCache();
+        }
+        // Now let's run update stats on tenant1View
+        try (Connection conn = getTenantConnection(tenantId1)) {
+            conn.createStatement().execute("UPDATE STATISTICS " + tenant1View);
+        }
+        // query tenant2 view
+        try (Connection conn = getTenantConnection(tenantId2)) {
+            sql = "SELECT * FROM " + tenant2View;
+
+            Estimate info = getByteRowEstimates(conn, sql, binds);
+            /*
+             * Because we ran update stats only for tenant1View, there is only partial guidepost
+             * info available for tenant2View.
+             */
+            assertEquals((Long) 1l, info.estimatedRows);
+            // ok now run update stats for tenant2 view
+            conn.createStatement().execute("UPDATE STATISTICS " + tenant2View);
+            /*
+             * And now, let's recheck our estimate info. We should have all the rows of view2
+             * available now.
+             */
+            info = getByteRowEstimates(conn, sql, binds);
+            assertEquals((Long) 6l, info.estimatedRows);
+        }
+    }
 }


[04/20] phoenix git commit: PHOENIX-4290 Full table scan performed for DELETE with table having immutable indexes

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/7a5b5da5/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
index 0d06f0a..c84e1d7 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
@@ -17,8 +17,6 @@
  */
 package org.apache.phoenix.compile;
 
-import static org.apache.phoenix.schema.PTable.QualifierEncodingScheme.NON_ENCODED_QUALIFIERS;
-
 import java.sql.SQLException;
 import java.sql.SQLFeatureNotSupportedException;
 import java.util.ArrayList;
@@ -66,6 +64,7 @@ import org.apache.phoenix.schema.ColumnNotFoundException;
 import org.apache.phoenix.schema.ColumnRef;
 import org.apache.phoenix.schema.FunctionNotFoundException;
 import org.apache.phoenix.schema.MetaDataClient;
+import org.apache.phoenix.schema.MetaDataEntityNotFoundException;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PColumnFamily;
 import org.apache.phoenix.schema.PColumnFamilyImpl;
@@ -73,9 +72,9 @@ import org.apache.phoenix.schema.PColumnImpl;
 import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PNameFactory;
 import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTable.ImmutableStorageScheme;
 import org.apache.phoenix.schema.PTable.IndexType;
 import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
-import org.apache.phoenix.schema.PTable.ImmutableStorageScheme;
 import org.apache.phoenix.schema.PTableImpl;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.PTableType;
@@ -871,7 +870,9 @@ public class FromCompiler {
                     TableRef tableRef = iterator.next();
                     try {
                         PColumnFamily columnFamily = tableRef.getTable().getColumnFamily(cfName);
-                        if (theColumnFamilyRef != null) { throw new TableNotFoundException(cfName); }
+                        if (columnFamily == null) { 
+                            throw new TableNotFoundException(cfName); 
+                        }
                         theColumnFamilyRef = new ColumnFamilyRef(tableRef, columnFamily);
                     } catch (ColumnFamilyNotFoundException e) {}
                 }
@@ -914,10 +915,42 @@ public class FromCompiler {
                     PColumn column = tableRef.getTable().getColumnForColumnName(colName);
                     return new ColumnRef(tableRef, column.getPosition());
                 } catch (TableNotFoundException e) {
-                    // Try using the tableName as a columnFamily reference instead
-                    ColumnFamilyRef cfRef = resolveColumnFamily(schemaName, tableName);
-                    PColumn column = cfRef.getFamily().getPColumnForColumnName(colName);
-                    return new ColumnRef(cfRef.getTableRef(), column.getPosition());
+                    TableRef theTableRef = null;
+                    PColumn theColumn = null;
+                    PColumnFamily theColumnFamily = null;
+                    if (schemaName != null) {
+                        try {
+                            // Try schemaName as the tableName and use tableName as column family name
+                            theTableRef = resolveTable(null, schemaName);
+                            theColumnFamily = theTableRef.getTable().getColumnFamily(tableName);
+                            theColumn = theColumnFamily.getPColumnForColumnName(colName);
+                        } catch (MetaDataEntityNotFoundException e2) {
+                        }
+                    } 
+                    if (theColumn == null) {
+                        // Try using the tableName as a columnFamily reference instead
+                        // and resolve column in each column family.
+                        Iterator<TableRef> iterator = tables.iterator();
+                        while (iterator.hasNext()) {
+                            TableRef tableRef = iterator.next();
+                            try {
+                                PColumnFamily columnFamily = tableRef.getTable().getColumnFamily(tableName);
+                                PColumn column = columnFamily.getPColumnForColumnName(colName);
+                                if (theColumn != null) {
+                                    throw new AmbiguousColumnException(colName);
+                                }
+                                theTableRef = tableRef;
+                                theColumnFamily = columnFamily;
+                                theColumn = column;
+                            } catch (MetaDataEntityNotFoundException e1) {
+                            }
+                        }
+                        if (theColumn == null) { 
+                            throw new ColumnNotFoundException(colName);
+                        }
+                    }
+                    ColumnFamilyRef cfRef = new ColumnFamilyRef(theTableRef, theColumnFamily);
+                    return new ColumnRef(cfRef.getTableRef(), theColumn.getPosition());
                 }
             }
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7a5b5da5/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
index 4ebca90..796dad0 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
@@ -153,7 +153,7 @@ public class TupleProjectionCompiler {
                 PTableType.PROJECTED, table.getIndexState(), table.getTimeStamp(), table.getSequenceNumber(),
                 table.getPKName(), table.getBucketNum(), projectedColumns, table.getParentSchemaName(),
                 table.getParentTableName(), table.getIndexes(), table.isImmutableRows(), Collections.<PName> emptyList(),
-                null, null, table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(),
+                table.getDefaultFamilyName(), table.getViewStatement(), table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(),
                 table.getViewIndexId(),
                 table.getIndexType(), table.rowKeyOrderOptimizable(), table.isTransactional(), table.getUpdateCacheFrequency(), 
                 table.getIndexDisableTimestamp(), table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter(), table.useStatsForParallelization());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7a5b5da5/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java b/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
index f25f7f1..cfeb212 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
@@ -244,7 +244,6 @@ public enum SQLExceptionCode {
     SET_UNSUPPORTED_PROP_ON_ALTER_TABLE(1025, "42Y83", "Unsupported property set in ALTER TABLE command."),
     CANNOT_ADD_NOT_NULLABLE_COLUMN(1038, "42Y84", "Only nullable columns may be added for a pre-existing table."),
     NO_MUTABLE_INDEXES(1026, "42Y85", "Mutable secondary indexes are only supported for HBase version " + MetaDataUtil.decodeHBaseVersionAsString(PhoenixDatabaseMetaData.MUTABLE_SI_VERSION_THRESHOLD) + " and above."),
-    INVALID_FILTER_ON_IMMUTABLE_ROWS(1027, "42Y86", "All columns referenced in a WHERE clause must be available in every index for a table with immutable rows."),
     INVALID_INDEX_STATE_TRANSITION(1028, "42Y87", "Invalid index state transition."),
     INVALID_MUTABLE_INDEX_CONFIG(1029, "42Y88", "Mutable secondary indexes must have the "
             + IndexManagementUtil.WAL_EDIT_CODEC_CLASS_KEY + " property set to "

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7a5b5da5/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
index bd0743c..b0974c6 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
@@ -190,7 +190,9 @@ public class MutationState implements SQLCloseable {
 
     public MutationState(TableRef table, Map<ImmutableBytesPtr,RowMutationState> mutations, long sizeOffset, long maxSize, long maxSizeBytes, PhoenixConnection connection) throws SQLException {
         this(maxSize, maxSizeBytes, connection, false, null, sizeOffset);
-        this.mutations.put(table, mutations);
+        if (!mutations.isEmpty()) {
+            this.mutations.put(table, mutations);
+        }
         this.numRows = mutations.size();
         throwIfTooBig();
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7a5b5da5/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java b/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
index 2c55bdf..dc26d5a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
@@ -366,7 +366,6 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
     
     private IndexMaintainer(final PTable dataTable, final PTable index, PhoenixConnection connection) {
         this(dataTable.getRowKeySchema(), dataTable.getBucketNum() != null);
-        assert(dataTable.getType() == PTableType.SYSTEM || dataTable.getType() == PTableType.TABLE || dataTable.getType() == PTableType.VIEW);
         this.rowKeyOrderOptimizable = index.rowKeyOrderOptimizable();
         this.isMultiTenant = dataTable.isMultiTenant();
         this.viewIndexId = index.getViewIndexId() == null ? null : MetaDataUtil.getViewIndexIdDataType().toBytes(index.getViewIndexId());
@@ -411,15 +410,14 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
         int nDataPKColumns = dataRowKeySchema.getFieldCount() - dataPosOffset;
         // For indexes on views, we need to remember which data columns are "constants"
         // These are the values in a VIEW where clause. For these, we don't put them in the
-        // index, as they're the same for every row in the index.
-        if (dataTable.getType() == PTableType.VIEW) {
-            List<PColumn>dataPKColumns = dataTable.getPKColumns();
-            for (int i = dataPosOffset; i < dataPKColumns.size(); i++) {
-                PColumn dataPKColumn = dataPKColumns.get(i);
-                if (dataPKColumn.getViewConstant() != null) {
-                    bitSet.set(i);
-                    nDataPKColumns--;
-                }
+        // index, as they're the same for every row in the index. The data table can be
+        // either a VIEW or PROJECTED
+        List<PColumn>dataPKColumns = dataTable.getPKColumns();
+        for (int i = dataPosOffset; i < dataPKColumns.size(); i++) {
+            PColumn dataPKColumn = dataPKColumns.get(i);
+            if (dataPKColumn.getViewConstant() != null) {
+                bitSet.set(i);
+                nDataPKColumns--;
             }
         }
         this.indexTableName = indexTableName;
@@ -543,11 +541,14 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
         for (int i = 0; i < index.getColumnFamilies().size(); i++) {
             PColumnFamily family = index.getColumnFamilies().get(i);
             for (PColumn indexColumn : family.getColumns()) {
-                PColumn dataColumn = IndexUtil.getDataColumn(dataTable, indexColumn.getName().getString());
-                byte[] dataColumnCq = dataColumn.getColumnQualifierBytes();
-                byte[] indexColumnCq = indexColumn.getColumnQualifierBytes();
-                this.coveredColumnsMap.put(new ColumnReference(dataColumn.getFamilyName().getBytes(), dataColumnCq), 
-                        new ColumnReference(indexColumn.getFamilyName().getBytes(), indexColumnCq));
+                PColumn dataColumn = IndexUtil.getDataColumnOrNull(dataTable, indexColumn.getName().getString());
+                // This can happen during deletion where we don't need covered columns
+                if (dataColumn != null) {
+                    byte[] dataColumnCq = dataColumn.getColumnQualifierBytes();
+                    byte[] indexColumnCq = indexColumn.getColumnQualifierBytes();
+                    this.coveredColumnsMap.put(new ColumnReference(dataColumn.getFamilyName().getBytes(), dataColumnCq), 
+                            new ColumnReference(indexColumn.getFamilyName().getBytes(), indexColumnCq));
+                }
             }
         }
         this.estimatedIndexRowKeyBytes = estimateIndexRowKeyByteSize(indexColByteSize);
@@ -758,8 +759,10 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
             int minLength = length - maxTrailingNulls;
             byte[] dataRowKey = stream.getBuffer();
             // Remove trailing nulls
-            while (length > minLength && dataRowKey[length-1] == QueryConstants.SEPARATOR_BYTE) {
+            int index = dataRowKeySchema.getFieldCount() - 1;
+            while (index >= 0 && !dataRowKeySchema.getField(index).getDataType().isFixedWidth() && length > minLength && dataRowKey[length-1] == QueryConstants.SEPARATOR_BYTE) {
                 length--;
+                index--;
             }
             // TODO: need to capture nDataSaltBuckets instead of just a boolean. For now,
             // we store this in nIndexSaltBuckets, as we only use this function for local indexes

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7a5b5da5/phoenix-core/src/main/java/org/apache/phoenix/optimize/QueryOptimizer.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/optimize/QueryOptimizer.java b/phoenix-core/src/main/java/org/apache/phoenix/optimize/QueryOptimizer.java
index ca7ff2c..b3df50b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/optimize/QueryOptimizer.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/optimize/QueryOptimizer.java
@@ -429,7 +429,7 @@ public class QueryOptimizer {
             
         });
         
-        return bestCandidates;
+        return stopAtBestPlan ? bestCandidates.subList(0, 1) : bestCandidates;
     }
 
     

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7a5b5da5/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 994b769..23b5161 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
@@ -295,6 +295,16 @@ public class PTableImpl implements PTable {
                 table.getIndexDisableTimestamp(), table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter(), table.useStatsForParallelization());
     }
     
+    public static PTableImpl makePTable(PTable table, PTableType type, Collection<PColumn> columns) throws SQLException {
+        return new PTableImpl(
+                table.getTenantId(), table.getSchemaName(), table.getTableName(), type, table.getIndexState(), table.getTimeStamp(),
+                table.getSequenceNumber(), table.getPKName(), table.getBucketNum(), columns, table.getParentSchemaName(), table.getParentTableName(),
+                table.getIndexes(), table.isImmutableRows(), table.getPhysicalNames(), table.getDefaultFamilyName(), table.getViewStatement(),
+                table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), table.getIndexType(),
+                table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.isTransactional(), table.getUpdateCacheFrequency(),
+                table.getIndexDisableTimestamp(), table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter(), table.useStatsForParallelization());
+    }
+
     public static PTableImpl makePTable(PTable table, Collection<PColumn> columns, PName defaultFamily) throws SQLException {
         return new PTableImpl(
                 table.getTenantId(), table.getSchemaName(), table.getTableName(), table.getType(), table.getIndexState(), table.getTimeStamp(),

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7a5b5da5/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 36fa011..cacf4c4 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
@@ -207,27 +207,35 @@ public class IndexUtil {
     }
     
     public static PColumn getDataColumn(PTable dataTable, String indexColumnName) {
+        PColumn column = getDataColumnOrNull(dataTable, indexColumnName);
+        if (column == null) {
+            throw new IllegalArgumentException("Could not find column \"" + SchemaUtil.getColumnName(getDataColumnFamilyName(indexColumnName), getDataColumnName(indexColumnName)) + " in " + dataTable);
+        }
+        return column;
+    }
+    
+    public static PColumn getDataColumnOrNull(PTable dataTable, String indexColumnName) {
         int pos = indexColumnName.indexOf(INDEX_COLUMN_NAME_SEP);
         if (pos < 0) {
-            throw new IllegalArgumentException("Could not find expected '" + INDEX_COLUMN_NAME_SEP +  "' separator in index column name of \"" + indexColumnName + "\"");
+            return null;
         }
         if (pos == 0) {
             try {
                 return dataTable.getPKColumn(indexColumnName.substring(1));
             } catch (ColumnNotFoundException e) {
-                throw new IllegalArgumentException("Could not find PK column \"" +  indexColumnName.substring(pos+1) + "\" in index column name of \"" + indexColumnName + "\"", e);
+                return null;
             }
         }
         PColumnFamily family;
         try {
             family = dataTable.getColumnFamily(getDataColumnFamilyName(indexColumnName));                
         } catch (ColumnFamilyNotFoundException e) {
-            throw new IllegalArgumentException("Could not find column family \"" +  indexColumnName.substring(0, pos) + "\" in index column name of \"" + indexColumnName + "\"", e);
+            return null;
         }
         try {
             return family.getPColumnForColumnName(indexColumnName.substring(pos+1));
         } catch (ColumnNotFoundException e) {
-            throw new IllegalArgumentException("Could not find column \"" +  indexColumnName.substring(pos+1) + "\" in index column name of \"" + indexColumnName + "\"", e);
+            return null;
         }
     }
     
@@ -686,7 +694,7 @@ public class IndexUtil {
     }
 
     public static byte[][] getViewConstants(PTable dataTable) {
-        if (dataTable.getType() != PTableType.VIEW) return null;
+        if (dataTable.getType() != PTableType.VIEW && dataTable.getType() != PTableType.PROJECTED) return null;
         int dataPosOffset = (dataTable.getBucketNum() != null ? 1 : 0) + (dataTable.isMultiTenant() ? 1 : 0);
         ImmutableBytesWritable ptr = new ImmutableBytesWritable();
         List<byte[]> viewConstants = new ArrayList<byte[]>();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7a5b5da5/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java b/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
index ca4be2f..b3c7dca 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
@@ -1235,33 +1235,6 @@ public class QueryCompilerTest extends BaseConnectionlessQueryTest {
     }
     
     @Test
-    public void testDeleteFromImmutableWithKV() throws Exception {
-        String ddl = "CREATE TABLE t (k1 VARCHAR, v1 VARCHAR, v2 VARCHAR CONSTRAINT pk PRIMARY KEY(k1)) immutable_rows=true";
-        String indexDDL = "CREATE INDEX i ON t (v1)";
-        Connection conn = DriverManager.getConnection(getUrl());
-        try {
-            conn.createStatement().execute(ddl);
-            assertImmutableRows(conn, "T", true);
-            conn.createStatement().execute(indexDDL);
-            assertImmutableRows(conn, "I", true);
-            conn.createStatement().execute("DELETE FROM t WHERE v2 = 'foo'");
-            fail();
-        } catch (SQLException e) {
-            assertEquals(SQLExceptionCode.INVALID_FILTER_ON_IMMUTABLE_ROWS.getErrorCode(), e.getErrorCode());
-        }
-        // Test with one index having the referenced key value column, but one not having it.
-        // Still should fail
-        try {
-            indexDDL = "CREATE INDEX i2 ON t (v2)";
-            conn.createStatement().execute(indexDDL);
-            conn.createStatement().execute("DELETE FROM t WHERE v2 = 'foo'");
-            fail();
-        } catch (SQLException e) {
-            assertEquals(SQLExceptionCode.INVALID_FILTER_ON_IMMUTABLE_ROWS.getErrorCode(), e.getErrorCode());
-        }
-    }
-    
-    @Test
     public void testInvalidNegativeArrayIndex() throws Exception {
         String query = "SELECT a_double_array[-20] FROM table_with_array";
         Connection conn = DriverManager.getConnection(getUrl());


[02/20] phoenix git commit: PHOENIX-4277 Treat delete markers consistently with puts for point-in-time scans

Posted by ja...@apache.org.
PHOENIX-4277 Treat delete markers consistently with puts for point-in-time scans


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

Branch: refs/heads/5.x-HBase-2.0
Commit: c39cd801601238ff171530a0ada1d1ea56d04edc
Parents: f513194
Author: James Taylor <jt...@salesforce.com>
Authored: Sun Oct 29 15:19:23 2017 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Thu Nov 9 12:41:46 2017 -0800

----------------------------------------------------------------------
 .../phoenix/end2end/PointInTimeQueryIT.java     |  2 +-
 .../hadoop/hbase/regionserver/ScanInfoUtil.java | 35 ++++++++++++++++++++
 .../coprocessor/BaseScannerRegionObserver.java  | 21 ++++++++++++
 .../apache/phoenix/util/TransactionUtil.java    |  7 ++--
 4 files changed, 62 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/c39cd801/phoenix-core/src/it/java/org/apache/phoenix/end2end/PointInTimeQueryIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PointInTimeQueryIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PointInTimeQueryIT.java
index c53e523..ed3e8a9 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PointInTimeQueryIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PointInTimeQueryIT.java
@@ -63,7 +63,7 @@ public class PointInTimeQueryIT extends BaseQueryIT {
     public PointInTimeQueryIT(String idxDdl, boolean columnEncoded)
             throws Exception {
         // These queries fail without KEEP_DELETED_CELLS=true
-        super(idxDdl, columnEncoded, true);
+        super(idxDdl, columnEncoded, false);
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c39cd801/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/ScanInfoUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/ScanInfoUtil.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/ScanInfoUtil.java
new file mode 100644
index 0000000..9885c78
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/ScanInfoUtil.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver;
+
+import org.apache.hadoop.hbase.KeepDeletedCells;
+
+public class ScanInfoUtil {
+    private ScanInfoUtil() {
+    }
+    
+    public static boolean isKeepDeletedCells(ScanInfo scanInfo) {
+        return scanInfo.getKeepDeletedCells() != KeepDeletedCells.FALSE;
+    }
+    
+    public static ScanInfo cloneScanInfoWithKeepDeletedCells(ScanInfo scanInfo) {
+        return new ScanInfo(scanInfo.getConfiguration(), scanInfo.getFamily(), Math.max(scanInfo.getMinVersions(), 1),
+                    scanInfo.getMaxVersions(), scanInfo.getTtl(), KeepDeletedCells.TRUE,
+                    scanInfo.getTimeToPurgeDeletes(), scanInfo.getComparator());
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c39cd801/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 95379a6..7c6df8f 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
@@ -19,9 +19,11 @@ package org.apache.phoenix.coprocessor;
 
 import java.io.IOException;
 import java.util.List;
+import java.util.NavigableSet;
 
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
@@ -29,10 +31,15 @@ import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.coprocessor.RegionObserver;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.io.TimeRange;
+import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.regionserver.ScanInfo;
+import org.apache.hadoop.hbase.regionserver.ScanInfoUtil;
 import org.apache.hadoop.hbase.regionserver.ScannerContext;
 import org.apache.hadoop.hbase.regionserver.ScannerContextUtil;
+import org.apache.hadoop.hbase.regionserver.Store;
+import org.apache.hadoop.hbase.regionserver.StoreScanner;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.htrace.Span;
 import org.apache.htrace.Trace;
@@ -47,6 +54,7 @@ import org.apache.phoenix.schema.types.PUnsignedTinyint;
 import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.ScanUtil;
 import org.apache.phoenix.util.ServerUtil;
+import org.apache.phoenix.util.TransactionUtil;
 
 
 abstract public class BaseScannerRegionObserver implements RegionObserver {
@@ -344,4 +352,17 @@ abstract public class BaseScannerRegionObserver implements RegionObserver {
                 dataRegion, indexMaintainer, null, viewConstants, null, null, projector, ptr, useQualiferAsListIndex);
     }
 
+    @Override
+    public KeyValueScanner preStoreScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c,
+        final Store store, final Scan scan, final NavigableSet<byte[]> targetCols,
+        final KeyValueScanner s) throws IOException {
+
+      if (scan.isRaw() || ScanInfoUtil.isKeepDeletedCells(store.getScanInfo()) || scan.getTimeRange().getMax() == HConstants.LATEST_TIMESTAMP || TransactionUtil.isTransactionalTimestamp(scan.getTimeRange().getMax())) {
+        return s;
+      }
+
+      ScanInfo scanInfo = ScanInfoUtil.cloneScanInfoWithKeepDeletedCells(store.getScanInfo());
+      return new StoreScanner(store, scanInfo, scan, targetCols,
+          c.getEnvironment().getRegion().getReadpoint(scan.getIsolationLevel()));
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c39cd801/phoenix-core/src/main/java/org/apache/phoenix/util/TransactionUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/TransactionUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/TransactionUtil.java
index f437087..fabbcc0 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/TransactionUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/TransactionUtil.java
@@ -24,8 +24,6 @@ import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.client.Table;
 import org.apache.phoenix.coprocessor.MetaDataProtocol.MetaDataMutationResult;
-import org.apache.phoenix.exception.SQLExceptionCode;
-import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.execute.MutationState;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.schema.PTable;
@@ -33,11 +31,16 @@ import org.apache.phoenix.transaction.PhoenixTransactionContext;
 import org.apache.phoenix.transaction.PhoenixTransactionalTable;
 import org.apache.phoenix.transaction.TephraTransactionTable;
 import org.apache.phoenix.transaction.TransactionFactory;
+import org.apache.tephra.util.TxUtils;
 
 public class TransactionUtil {
     private TransactionUtil() {
     }
     
+    public static boolean isTransactionalTimestamp(long ts) {
+        return !TxUtils.isPreExistingVersion(ts);
+    }
+    
     public static boolean isDelete(Cell cell) {
         return (CellUtil.matchingValue(cell, HConstants.EMPTY_BYTE_ARRAY));
     }


[06/20] phoenix git commit: PHOENIX-4287 Incorrect aggregate query results when stats are disable for parallelization

Posted by ja...@apache.org.
PHOENIX-4287 Incorrect aggregate query results when stats are disable for parallelization


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

Branch: refs/heads/5.x-HBase-2.0
Commit: 8a19ae1c4b975a4dd0c6f535819c9d2e94ab108f
Parents: 7a5b5da
Author: Samarth Jain <sa...@apache.org>
Authored: Tue Oct 31 10:12:22 2017 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Thu Nov 9 12:42:58 2017 -0800

----------------------------------------------------------------------
 .../end2end/ExplainPlanWithStatsEnabledIT.java  | 209 ++++++++++++++++++-
 .../phoenix/iterate/BaseResultIterators.java    |  55 +++--
 2 files changed, 246 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/8a19ae1c/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
index 62538af..931c398 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
@@ -20,6 +20,7 @@ package org.apache.phoenix.end2end;
 import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_USE_STATS_FOR_PARALLELIZATION;
 import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 import java.sql.Connection;
@@ -387,11 +388,8 @@ public class ExplainPlanWithStatsEnabledIT extends ParallelStatsEnabledIT {
     @Test
     public void testBytesRowsForSelectOnTenantViews() throws Exception {
         String tenant1View = generateUniqueName();
-        ;
         String tenant2View = generateUniqueName();
-        ;
         String tenant3View = generateUniqueName();
-        ;
         String multiTenantBaseTable = generateUniqueName();
         String tenant1 = "tenant1";
         String tenant2 = "tenant2";
@@ -504,6 +502,211 @@ public class ExplainPlanWithStatsEnabledIT extends ParallelStatsEnabledIT {
         }
     }
 
+    @Test // See https://issues.apache.org/jira/browse/PHOENIX-4287
+    public void testEstimatesForAggregateQueries() throws Exception {
+        String tableName = generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            int guidePostWidth = 20;
+            String ddl =
+                    "CREATE TABLE " + tableName + " (k INTEGER PRIMARY KEY, a bigint, b bigint)"
+                            + " GUIDE_POSTS_WIDTH=" + guidePostWidth
+                            + ", USE_STATS_FOR_PARALLELIZATION=false";
+            byte[][] splits =
+                    new byte[][] { Bytes.toBytes(102), Bytes.toBytes(105), Bytes.toBytes(108) };
+            BaseTest.createTestTable(getUrl(), ddl, splits, null);
+            conn.createStatement().execute("upsert into " + tableName + " values (100,1,3)");
+            conn.createStatement().execute("upsert into " + tableName + " values (101,2,4)");
+            conn.createStatement().execute("upsert into " + tableName + " values (102,2,4)");
+            conn.createStatement().execute("upsert into " + tableName + " values (103,2,4)");
+            conn.createStatement().execute("upsert into " + tableName + " values (104,2,4)");
+            conn.createStatement().execute("upsert into " + tableName + " values (105,2,4)");
+            conn.createStatement().execute("upsert into " + tableName + " values (106,2,4)");
+            conn.createStatement().execute("upsert into " + tableName + " values (107,2,4)");
+            conn.createStatement().execute("upsert into " + tableName + " values (108,2,4)");
+            conn.createStatement().execute("upsert into " + tableName + " values (109,2,4)");
+            conn.commit();
+            conn.createStatement().execute("UPDATE STATISTICS " + tableName + "");
+        }
+        List<Object> binds = Lists.newArrayList();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String sql = "SELECT COUNT(*) " + " FROM " + tableName;
+            ResultSet rs = conn.createStatement().executeQuery(sql);
+            assertTrue(rs.next());
+            assertEquals(10, rs.getInt(1));
+            Estimate info = getByteRowEstimates(conn, sql, binds);
+            assertEquals((Long) 10l, info.getEstimatedRows());
+            assertTrue(info.getEstimateInfoTs() > 0);
+
+            // Now let's make sure that when using stats for parallelization, our estimates
+            // and query results stay the same
+            conn.createStatement().execute(
+                "ALTER TABLE " + tableName + " SET USE_STATS_FOR_PARALLELIZATION=true");
+            rs = conn.createStatement().executeQuery(sql);
+            assertTrue(rs.next());
+            assertEquals(10, rs.getInt(1));
+            info = getByteRowEstimates(conn, sql, binds);
+            assertEquals((Long) 10l, info.getEstimatedRows());
+            assertTrue(info.getEstimateInfoTs() > 0);
+        }
+    }
+
+    @Test
+    public void testSelectQueriesWithStatsForParallelizationOff() throws Exception {
+        testSelectQueriesWithFilters(false);
+    }
+
+    @Test
+    public void testSelectQueriesWithStatsForParallelizationOn() throws Exception {
+        testSelectQueriesWithFilters(true);
+    }
+
+    private void testSelectQueriesWithFilters(boolean useStatsForParallelization) throws Exception {
+        String tableName = generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            int guidePostWidth = 20;
+            String ddl =
+                    "CREATE TABLE " + tableName + " (k INTEGER PRIMARY KEY, a bigint, b bigint)"
+                            + " GUIDE_POSTS_WIDTH=" + guidePostWidth
+                            + ", USE_STATS_FOR_PARALLELIZATION=" + useStatsForParallelization;
+            byte[][] splits =
+                    new byte[][] { Bytes.toBytes(102), Bytes.toBytes(105), Bytes.toBytes(108) };
+            BaseTest.createTestTable(getUrl(), ddl, splits, null);
+            conn.createStatement().execute("upsert into " + tableName + " values (100,100,3)");
+            conn.createStatement().execute("upsert into " + tableName + " values (101,101,4)");
+            conn.createStatement().execute("upsert into " + tableName + " values (102,102,4)");
+            conn.createStatement().execute("upsert into " + tableName + " values (103,103,4)");
+            conn.createStatement().execute("upsert into " + tableName + " values (104,104,4)");
+            conn.createStatement().execute("upsert into " + tableName + " values (105,105,4)");
+            conn.createStatement().execute("upsert into " + tableName + " values (106,106,4)");
+            conn.createStatement().execute("upsert into " + tableName + " values (107,107,4)");
+            conn.createStatement().execute("upsert into " + tableName + " values (108,108,4)");
+            conn.createStatement().execute("upsert into " + tableName + " values (109,109,4)");
+            conn.commit();
+            conn.createStatement().execute("UPDATE STATISTICS " + tableName + "");
+        }
+        List<Object> binds = Lists.newArrayList();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            // query whose start key is before any data
+            String sql = "SELECT a FROM " + tableName + " WHERE K >= 99";
+            ResultSet rs = conn.createStatement().executeQuery(sql);
+            int i = 0;
+            int numRows = 10;
+            while (rs.next()) {
+                assertEquals(100 + i, rs.getInt(1));
+                i++;
+            }
+            assertEquals(numRows, i);
+            Estimate info = getByteRowEstimates(conn, sql, binds);
+            assertEquals((Long) 10l, info.getEstimatedRows());
+            assertEquals((Long) 930l, info.getEstimatedBytes());
+            assertTrue(info.getEstimateInfoTs() > 0);
+
+            // query whose start key is after any data
+            sql = "SELECT a FROM " + tableName + " WHERE K >= 110";
+            rs = conn.createStatement().executeQuery(sql);
+            assertFalse(rs.next());
+            info = getByteRowEstimates(conn, sql, binds);
+            assertEquals((Long) 0l, info.getEstimatedRows());
+            assertEquals((Long) 0l, info.getEstimatedBytes());
+            assertTrue(info.getEstimateInfoTs() > 0);
+
+            // Query whose end key is before any data
+            sql = "SELECT a FROM " + tableName + " WHERE K <= 98";
+            rs = conn.createStatement().executeQuery(sql);
+            assertFalse(rs.next());
+            info = getByteRowEstimates(conn, sql, binds);
+            assertEquals((Long) 0l, info.getEstimatedRows());
+            assertEquals((Long) 0l, info.getEstimatedBytes());
+            assertTrue(info.getEstimateInfoTs() > 0);
+
+            // Query whose end key is after any data. In this case, we return the estimate as
+            // scanning all the guide posts.
+            sql = "SELECT a FROM " + tableName + " WHERE K <= 110";
+            rs = conn.createStatement().executeQuery(sql);
+            i = 0;
+            numRows = 10;
+            while (rs.next()) {
+                assertEquals(100 + i, rs.getInt(1));
+                i++;
+            }
+            assertEquals(numRows, i);
+            info = getByteRowEstimates(conn, sql, binds);
+            assertEquals((Long) 10l, info.getEstimatedRows());
+            assertEquals((Long) 930l, info.getEstimatedBytes());
+            assertTrue(info.getEstimateInfoTs() > 0);
+
+            // Query whose start key and end key is before any data. In this case, we return the
+            // estimate as
+            // scanning the first guide post
+            sql = "SELECT a FROM " + tableName + " WHERE K <= 90 AND K >= 80";
+            rs = conn.createStatement().executeQuery(sql);
+            assertFalse(rs.next());
+            info = getByteRowEstimates(conn, sql, binds);
+            assertEquals((Long) 0l, info.getEstimatedRows());
+            assertEquals((Long) 0l, info.getEstimatedBytes());
+            assertTrue(info.getEstimateInfoTs() > 0);
+
+            // Query whose start key and end key is after any data. In this case, we return the
+            // estimate as
+            // scanning no guide post
+            sql = "SELECT a FROM " + tableName + " WHERE K <= 130 AND K >= 120";
+            rs = conn.createStatement().executeQuery(sql);
+            assertFalse(rs.next());
+            info = getByteRowEstimates(conn, sql, binds);
+            assertEquals((Long) 0l, info.getEstimatedRows());
+            assertEquals((Long) 0l, info.getEstimatedBytes());
+            assertTrue(info.getEstimateInfoTs() > 0);
+
+            // Query whose start key is before and end key is between data. In this case, we return
+            // the estimate as
+            // scanning no guide post
+            sql = "SELECT a FROM " + tableName + " WHERE K <= 102 AND K >= 90";
+            rs = conn.createStatement().executeQuery(sql);
+            i = 0;
+            numRows = 3;
+            while (rs.next()) {
+                assertEquals(100 + i, rs.getInt(1));
+                i++;
+            }
+            info = getByteRowEstimates(conn, sql, binds);
+            // Depending on the guidepost boundary, this estimate
+            // can be slightly off. It's called estimate for a reason.
+            assertEquals((Long) 4l, info.getEstimatedRows());
+            assertEquals((Long) 330l, info.getEstimatedBytes());
+            assertTrue(info.getEstimateInfoTs() > 0);
+            // Query whose start key is between and end key is after data.
+            sql = "SELECT a FROM " + tableName + " WHERE K <= 120 AND K >= 100";
+            rs = conn.createStatement().executeQuery(sql);
+            i = 0;
+            numRows = 10;
+            while (rs.next()) {
+                assertEquals(100 + i, rs.getInt(1));
+                i++;
+            }
+            info = getByteRowEstimates(conn, sql, binds);
+            // Depending on the guidepost boundary, this estimate
+            // can be slightly off. It's called estimate for a reason.
+            assertEquals((Long) 9l, info.getEstimatedRows());
+            assertEquals((Long) 900l, info.getEstimatedBytes());
+            assertTrue(info.getEstimateInfoTs() > 0);
+            // Query whose start key and end key are both between data.
+            sql = "SELECT a FROM " + tableName + " WHERE K <= 109 AND K >= 100";
+            rs = conn.createStatement().executeQuery(sql);
+            i = 0;
+            numRows = 10;
+            while (rs.next()) {
+                assertEquals(100 + i, rs.getInt(1));
+                i++;
+            }
+            info = getByteRowEstimates(conn, sql, binds);
+            // Depending on the guidepost boundary, this estimate
+            // can be slightly off. It's called estimate for a reason.
+            assertEquals((Long) 9l, info.getEstimatedRows());
+            assertEquals((Long) 900l, info.getEstimatedBytes());
+            assertTrue(info.getEstimateInfoTs() > 0);
+        }
+    }
+
     private static void createMultitenantTableAndViews(String tenant1View, String tenant2View,
             String tenant3View, String tenant1, String tenant2, String tenant3,
             String multiTenantTable, MyClock clock) throws SQLException {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8a19ae1c/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
index 9bf9573..dce8de1 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
@@ -35,6 +35,7 @@ import java.io.ByteArrayInputStream;
 import java.io.DataInput;
 import java.io.DataInputStream;
 import java.io.EOFException;
+import java.io.IOException;
 import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.BitSet;
@@ -585,15 +586,29 @@ public abstract class BaseResultIterators extends ExplainTable implements Result
         return context.getConnection().getQueryServices().getTableStats(key);
     }
 
-    private List<Scan> addNewScan(List<List<Scan>> parallelScans, List<Scan> scans, Scan scan, byte[] startKey, boolean crossedRegionBoundary, HRegionLocation regionLocation) {
+    private List<Scan> addNewScan(List<List<Scan>> parallelScans, List<Scan> scans, Scan scan,
+            byte[] startKey, boolean crossedRegionBoundary, HRegionLocation regionLocation,
+            GuidePostEstimate estimate, Long gpsRows, Long gpsBytes) {
         boolean startNewScan = scanGrouper.shouldStartNewScan(plan, scans, startKey, crossedRegionBoundary);
         if (scan != null) {
             if (regionLocation.getServerName() != null) {
                 scan.setAttribute(BaseScannerRegionObserver.SCAN_REGION_SERVER, regionLocation.getServerName().getVersionedBytes());
             }
-        	scans.add(scan);
+            if (useStatsForParallelization || crossedRegionBoundary) {
+                scans.add(scan);
+            }
+            if (estimate != null && gpsRows != null) {
+                estimate.rowsEstimate += gpsRows;
+            }
+            if (estimate != null && gpsBytes != null) {
+                estimate.bytesEstimate += gpsBytes;
+            }
         }
-        if (startNewScan && !scans.isEmpty()) {
+        if (startNewScan && !scans.isEmpty() && useStatsForParallelization) {
+            /*
+             * Note that even if region boundary was crossed, if we are not using stats for
+             * parallelization, nothing gets added to the parallel scans.
+             */
             parallelScans.add(scans);
             scans = Lists.newArrayListWithExpectedSize(1);
         }
@@ -653,7 +668,7 @@ public abstract class BaseResultIterators extends ExplainTable implements Result
                     newScan.setStopRow(regionInfo.getEndKey());
                 }
             }
-            scans = addNewScan(parallelScans, scans, newScan, endKey, true, regionLocation);
+            scans = addNewScan(parallelScans, scans, newScan, endKey, true, regionLocation, null, null, null);
             regionIndex++;
         }
         if (!scans.isEmpty()) { // Add any remaining scans
@@ -662,6 +677,11 @@ public abstract class BaseResultIterators extends ExplainTable implements Result
         return parallelScans;
     }
 
+    private static class GuidePostEstimate {
+        private long bytesEstimate;
+        private long rowsEstimate;
+    }
+
     /**
      * Compute the list of parallel scans to run for a given query. The inner scans
      * may be concatenated together directly, while the other ones may need to be
@@ -721,8 +741,7 @@ public abstract class BaseResultIterators extends ExplainTable implements Result
         DataInput input = null;
         PrefixByteDecoder decoder = null;
         int guideIndex = 0;
-        long estimatedRows = 0;
-        long estimatedSize = 0;
+        GuidePostEstimate estimates = new GuidePostEstimate();
         long estimateTs = Long.MAX_VALUE;
         long minGuidePostTimestamp = Long.MAX_VALUE;
         try {
@@ -763,6 +782,7 @@ public abstract class BaseResultIterators extends ExplainTable implements Result
                     endRegionKey = regionInfo.getEndKey();
                     keyOffset = ScanUtil.getRowKeyOffset(regionInfo.getStartKey(), endRegionKey);
                 }
+                byte[] initialKeyBytes = currentKeyBytes;
                 while (intersectWithGuidePosts && (endKey.length == 0 || currentGuidePost.compareTo(endKey) <= 0)) {
                     Scan newScan = scanRanges.intersectScan(scan, currentKeyBytes, currentGuidePostBytes, keyOffset,
                         false);
@@ -770,12 +790,11 @@ public abstract class BaseResultIterators extends ExplainTable implements Result
                         ScanUtil.setLocalIndexAttributes(newScan, keyOffset,
                             regionInfo.getStartKey(), regionInfo.getEndKey(),
                             newScan.getStartRow(), newScan.getStopRow());
-                        estimatedRows += gps.getRowCounts()[guideIndex];
-                        estimatedSize += gps.getByteCounts()[guideIndex];
-                    }
-                    if (useStatsForParallelization) {
-                        scans = addNewScan(parallelScans, scans, newScan, currentGuidePostBytes, false, regionLocation);
                     }
+                    scans =
+                            addNewScan(parallelScans, scans, newScan, currentGuidePostBytes, false,
+                                regionLocation, estimates, gps.getRowCounts()[guideIndex],
+                                gps.getByteCounts()[guideIndex]);
                     currentKeyBytes = currentGuidePostBytes;
                     try {
                         currentGuidePost = PrefixByteCodec.decode(decoder, input);
@@ -794,12 +813,19 @@ public abstract class BaseResultIterators extends ExplainTable implements Result
                         intersectWithGuidePosts = false;
                     }
                 }
+                if (!useStatsForParallelization) {
+                    /*
+                     * If we are not using stats for generating parallel scans, we need to reset the
+                     * currentKey back to what it was at the beginning of the loop.
+                     */
+                    currentKeyBytes = initialKeyBytes;
+                }
                 Scan newScan = scanRanges.intersectScan(scan, currentKeyBytes, endKey, keyOffset, true);
                 if(newScan != null) {
                     ScanUtil.setLocalIndexAttributes(newScan, keyOffset, regionInfo.getStartKey(),
                         regionInfo.getEndKey(), newScan.getStartRow(), newScan.getStopRow());
                 }
-                scans = addNewScan(parallelScans, scans, newScan, endKey, true, regionLocation);
+                scans = addNewScan(parallelScans, scans, newScan, endKey, true, regionLocation, null, null, null);
                 currentKeyBytes = endKey;
                 regionIndex++;
             }
@@ -814,8 +840,8 @@ public abstract class BaseResultIterators extends ExplainTable implements Result
                 this.estimatedSize = gps.getByteCounts()[0];
                 this.estimateInfoTimestamp = gps.getGuidePostTimestamps()[0];
             } else if (hasGuidePosts) {
-                this.estimatedRows = estimatedRows;
-                this.estimatedSize = estimatedSize;
+                this.estimatedRows = estimates.rowsEstimate;
+                this.estimatedSize = estimates.bytesEstimate;
                 this.estimateInfoTimestamp = estimateTs;
             } else {
                 this.estimatedRows = null;
@@ -828,7 +854,6 @@ public abstract class BaseResultIterators extends ExplainTable implements Result
         } finally {
             if (stream != null) Closeables.closeQuietly(stream);
         }
-        
         sampleScans(parallelScans,this.plan.getStatement().getTableSamplingRate());
         return parallelScans;
     }


[11/20] phoenix git commit: PHOENIX-4332 Indexes should inherit guide post width of the base data table

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/59e49f78/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/DefaultStatisticsCollector.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/DefaultStatisticsCollector.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/DefaultStatisticsCollector.java
index d82c66f..4e4978c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/DefaultStatisticsCollector.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/DefaultStatisticsCollector.java
@@ -18,6 +18,8 @@
 package org.apache.phoenix.schema.stats;
 
 import java.io.IOException;
+import java.sql.Connection;
+import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
@@ -45,15 +47,22 @@ import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.PName;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTable.IndexType;
+import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.types.PInteger;
 import org.apache.phoenix.schema.types.PLong;
 import org.apache.phoenix.util.EnvironmentEdgeManager;
 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.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Maps;
 
 /**
@@ -75,6 +84,7 @@ class DefaultStatisticsCollector implements StatisticsCollector {
     private ImmutableBytesWritable currentRow;
     private final long clientTimeStamp;
     private final String tableName;
+    private final boolean isViewIndexTable;
 
     DefaultStatisticsCollector(RegionCoprocessorEnvironment env, String tableName, long clientTimeStamp, byte[] family,
             byte[] gp_width_bytes, byte[] gp_per_region_bytes) throws IOException {
@@ -95,6 +105,9 @@ class DefaultStatisticsCollector implements StatisticsCollector {
         // since there's no row representing those in SYSTEM.CATALOG.
         if (MetaDataUtil.isViewIndex(tableName)) {
             pName = MetaDataUtil.getViewIndexUserTableName(tableName);
+            isViewIndexTable = true;
+        } else {
+            isViewIndexTable = false;
         }
         ptableKey = SchemaUtil.getTableKeyFromFullName(pName);
         this.clientTimeStamp = clientTimeStamp;
@@ -109,7 +122,7 @@ class DefaultStatisticsCollector implements StatisticsCollector {
         }
     }
     
-    private void initGuidepostDepth() throws IOException {
+    private void initGuidepostDepth() throws IOException, ClassNotFoundException, SQLException {
         // First check is if guidepost info set on statement itself
         if (guidePostPerRegionBytes != null || guidePostWidthBytes != null) {
             int guidepostPerRegion = 0;
@@ -135,6 +148,38 @@ class DefaultStatisticsCollector implements StatisticsCollector {
                 if (!result.isEmpty()) {
                     Cell cell = result.listCells().get(0);
                     guidepostWidth = PLong.INSTANCE.getCodec().decodeLong(cell.getValueArray(), cell.getValueOffset(), SortOrder.getDefault());
+                } else if (!isViewIndexTable) {
+                    /*
+                     * The table we are collecting stats for is potentially a base table, or local
+                     * index or a global index. For view indexes, we rely on the the guide post
+                     * width column in the parent data table's metadata which we already tried
+                     * retrieving above.
+                     */
+                    try (Connection conn =
+                            QueryUtil.getConnectionOnServer(env.getConfiguration())) {
+                        PTable table = PhoenixRuntime.getTable(conn, tableName);
+                        if (table.getType() == PTableType.INDEX
+                                && table.getIndexType() == IndexType.GLOBAL) {
+                            /*
+                             * For global indexes, we need to get the parentName first and then
+                             * fetch guide post width configured for the parent table.
+                             */
+                            PName parentName = table.getParentName();
+                            byte[] parentKey =
+                                    SchemaUtil.getTableKeyFromFullName(parentName.getString());
+                            get = new Get(parentKey);
+                            get.addColumn(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
+                                PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH_BYTES);
+                            result = htable.get(get);
+                            if (!result.isEmpty()) {
+                                Cell cell = result.listCells().get(0);
+                                guidepostWidth =
+                                        PLong.INSTANCE.getCodec().decodeLong(cell.getValueArray(),
+                                            cell.getValueOffset(), SortOrder.getDefault());
+                            }
+                        }
+                    }
+
                 }
             } finally {
                 if (htable != null) {
@@ -318,7 +363,11 @@ class DefaultStatisticsCollector implements StatisticsCollector {
 
     @Override
     public void init() throws IOException {
-        initGuidepostDepth();
+        try {
+            initGuidepostDepth();
+        } catch (ClassNotFoundException | SQLException e) {
+            throw new IOException("Unable to initialize the guide post depth", e);
+        }
         this.statsWriter = StatisticsWriter.newWriter(env, tableName, clientTimeStamp, guidePostDepth);
     }
 
@@ -331,4 +380,9 @@ class DefaultStatisticsCollector implements StatisticsCollector {
         return null;
     }
 
+    @VisibleForTesting // Don't call this method anywhere else
+    public long getGuidePostDepth() {
+        return guidePostDepth;
+    }
+
 }


[08/20] phoenix git commit: PHOENIX-4335 System catalog snapshot created each time a new connection is created

Posted by ja...@apache.org.
PHOENIX-4335 System catalog snapshot created each time a new connection is created


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

Branch: refs/heads/5.x-HBase-2.0
Commit: 2a67137db102af7bf7adf6a42863b110343f64d3
Parents: 79d80f1
Author: James Taylor <jt...@salesforce.com>
Authored: Tue Oct 31 15:55:03 2017 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Thu Nov 9 12:43:41 2017 -0800

----------------------------------------------------------------------
 .../phoenix/end2end/SystemCatalogUpgradeIT.java | 121 +++++++++++++++++++
 .../phoenix/coprocessor/MetaDataProtocol.java   |  12 +-
 .../query/ConnectionQueryServicesImpl.java      |  39 ++++--
 .../java/org/apache/phoenix/query/BaseTest.java |  35 ++++--
 4 files changed, 190 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a67137d/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogUpgradeIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogUpgradeIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogUpgradeIT.java
new file mode 100644
index 0000000..e5b1d6e
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogUpgradeIT.java
@@ -0,0 +1,121 @@
+/*
+ * 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 static org.junit.Assert.assertTrue;
+
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.Map;
+import java.util.Properties;
+
+import org.apache.phoenix.coprocessor.MetaDataProtocol;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver.ConnectionInfo;
+import org.apache.phoenix.jdbc.PhoenixTestDriver;
+import org.apache.phoenix.query.BaseTest;
+import org.apache.phoenix.query.ConnectionQueryServices;
+import org.apache.phoenix.query.ConnectionQueryServicesImpl;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesTestImpl;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.google.common.collect.Maps;
+
+public class SystemCatalogUpgradeIT extends BaseTest {
+    private static boolean reinitialize;
+    private static int countUpgradeAttempts;
+    private static long systemTableVersion = MetaDataProtocol.getPriorVersion();
+    
+    private static class PhoenixUpgradeCountingServices extends ConnectionQueryServicesImpl {
+        public PhoenixUpgradeCountingServices(QueryServices services, ConnectionInfo connectionInfo, Properties info) {
+            super(services, connectionInfo, info);
+        }
+        
+        @Override
+        protected void setUpgradeRequired() {
+            super.setUpgradeRequired();
+            countUpgradeAttempts++;
+        }
+        
+        @Override
+        protected long getSystemTableVersion() {
+            return systemTableVersion;
+        }
+        
+        @Override
+        protected boolean isInitialized() {
+            return !reinitialize && super.isInitialized();
+        }
+    }
+    
+    public static class PhoenixUpgradeCountingDriver extends PhoenixTestDriver {
+        private ConnectionQueryServices cqs;
+        private final ReadOnlyProps overrideProps;
+        
+        public PhoenixUpgradeCountingDriver(ReadOnlyProps props) {
+            overrideProps = props;
+        }
+        
+        @Override
+        public boolean acceptsURL(String url) throws SQLException {
+            return true;
+        }
+        
+        @Override // public for testing
+        public synchronized ConnectionQueryServices getConnectionQueryServices(String url, Properties info) throws SQLException {
+            if (cqs == null) {
+                cqs = new PhoenixUpgradeCountingServices(new QueryServicesTestImpl(getDefaultProps(), overrideProps), ConnectionInfo.create(url), info);
+                cqs.init(url, info);
+            } else if (reinitialize) {
+                cqs.init(url, info);
+                reinitialize = false;
+            }
+            return cqs;
+        }
+    }
+    
+    @BeforeClass
+    public static void doSetup() throws Exception {
+        Map<String, String> props = Maps.newConcurrentMap();
+        props.put(BaseTest.DRIVER_CLASS_NAME_ATTRIB, PhoenixUpgradeCountingDriver.class.getName());
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+    }
+
+    @Test
+    public void testUpgradeOnlyHappensOnce() throws Exception {
+        ConnectionQueryServices services = DriverManager.getConnection(getUrl()).unwrap(PhoenixConnection.class).getQueryServices();
+        assertTrue(services instanceof PhoenixUpgradeCountingServices);
+        // Check if the timestamp version is changing between the current version and prior version
+        boolean wasTimestampChanged = systemTableVersion != MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP;
+        reinitialize = true;
+        systemTableVersion = MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP;
+        DriverManager.getConnection(getUrl());
+        // Confirm that if the timestamp changed, that an upgrade was performed (and that if it
+        // didn't, that an upgrade wasn't attempted).
+        assertEquals(wasTimestampChanged ? 1 : 0, countUpgradeAttempts);
+        // Confirm that another connection does not increase the number of times upgrade was attempted
+        DriverManager.getConnection(getUrl());
+        assertEquals(wasTimestampChanged ? 1 : 0, countUpgradeAttempts);
+    }
+
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a67137d/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
index 655068d..c4ecc3f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
@@ -19,6 +19,7 @@ package org.apache.phoenix.coprocessor;
 
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.Iterator;
 import java.util.List;
 import java.util.NavigableMap;
 import java.util.TreeMap;
@@ -89,7 +90,8 @@ public abstract class MetaDataProtocol extends MetaDataService {
     public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_9_0 = MIN_TABLE_TIMESTAMP + 20;
     public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_10_0 = MIN_TABLE_TIMESTAMP + 25;
     public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_11_0 = MIN_TABLE_TIMESTAMP + 27;
-    public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_12_0 = MIN_TABLE_TIMESTAMP + 28;
+    // Since there's no upgrade code, keep the version the same as the previous version
+    public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_12_0 = MIN_SYSTEM_TABLE_TIMESTAMP_4_11_0;
     // MIN_SYSTEM_TABLE_TIMESTAMP needs to be set to the max of all the MIN_SYSTEM_TABLE_TIMESTAMP_* constants
     public static final long MIN_SYSTEM_TABLE_TIMESTAMP = MIN_SYSTEM_TABLE_TIMESTAMP_4_12_0;
     
@@ -431,6 +433,14 @@ public abstract class MetaDataProtocol extends MetaDataService {
         }
     }
   
+    public static long getPriorVersion() {
+        Iterator<Long> iterator = TIMESTAMP_VERSION_MAP.descendingKeySet().iterator();
+        if (!iterator.hasNext()) {
+            return -1;
+        }
+        return iterator.next();
+    }
+    
     public static String getVersion(long serverTimestamp) {
         /*
          * It is possible that when clients are trying to run upgrades concurrently, we could be at an intermediate

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a67137d/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 4c60aa9..6e9b40e 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
@@ -2356,13 +2356,38 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         return addColumn(oldMetaConnection, tableName, timestamp, columns, true);
     }
 
+    // Available for testing
+    protected long getSystemTableVersion() {
+        return MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP;
+    }
+    
+    // Available for testing
+    protected void setUpgradeRequired() {
+        this.upgradeRequired.set(true);
+    }
+    
+    // Available for testing
+    protected boolean isInitialized() {
+        return initialized;
+    }
+    
+    // Available for testing
+    protected void setInitialized(boolean isInitialized) {
+        initialized = isInitialized;
+    }
+
+    // Available for testing
+    protected String getSystemCatalogDML() {
+        return QueryConstants.CREATE_TABLE_METADATA;
+    }
+
     @Override
     public void init(final String url, final Properties props) throws SQLException {
         try {
             PhoenixContextExecutor.call(new Callable<Void>() {
                 @Override
                 public Void call() throws Exception {
-                    if (initialized) {
+                    if (isInitialized()) {
                         if (initializationException != null) {
                             // Throw previous initialization exception, as we won't resuse this instance
                             throw initializationException;
@@ -2370,7 +2395,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         return null;
                     }
                     synchronized (ConnectionQueryServicesImpl.this) {
-                        if (initialized) {
+                        if (isInitialized()) {
                             if (initializationException != null) {
                                 // Throw previous initialization exception, as we won't resuse this instance
                                 throw initializationException;
@@ -2412,7 +2437,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                             }
                             Properties scnProps = PropertiesUtil.deepCopy(props);
                             scnProps.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB,
-                                    Long.toString(MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP));
+                                    Long.toString(getSystemTableVersion()));
                             scnProps.remove(PhoenixRuntime.TENANT_ID_ATTRIB);
                             String globalUrl = JDBCUtil.removeProperty(url, PhoenixRuntime.TENANT_ID_ATTRIB);
                             try (HBaseAdmin hBaseAdmin = getAdmin();
@@ -2420,7 +2445,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                          scnProps, newEmptyMetaData())) {
                                 try {
                                     metaConnection.setRunningUpgrade(true);
-                                    metaConnection.createStatement().executeUpdate(QueryConstants.CREATE_TABLE_METADATA);
+                                    metaConnection.createStatement().executeUpdate(getSystemCatalogDML());
                                 } catch (NewerTableAlreadyExistsException ignore) {
                                     // Ignore, as this will happen if the SYSTEM.CATALOG already exists at this fixed
                                     // timestamp. A TableAlreadyExistsException is not thrown, since the table only exists
@@ -2428,7 +2453,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                 } catch (TableAlreadyExistsException e) {
                                     long currentServerSideTableTimeStamp = e.getTable().getTimeStamp();
                                     if (currentServerSideTableTimeStamp < MIN_SYSTEM_TABLE_TIMESTAMP) {
-                                        ConnectionQueryServicesImpl.this.upgradeRequired.set(true);
+                                        setUpgradeRequired();
                                     }
                                 } catch (PhoenixIOException e) {
                                     if (!Iterables.isEmpty(Iterables.filter(Throwables.getCausalChain(e), AccessDeniedException.class))) {
@@ -2487,7 +2512,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                         throw initializationException;
                                     }
                                 } finally {
-                                    initialized = true;
+                                    setInitialized(true);
                                 }
                             }
                         }
@@ -2570,7 +2595,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE);
         boolean snapshotCreated = false;
         try {
-            if (!ConnectionQueryServicesImpl.this.upgradeRequired.get()) {
+            if (!isUpgradeRequired()) {
                 throw new UpgradeNotRequiredException();
             }
             Properties scnProps = PropertiesUtil.deepCopy(props);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a67137d/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 8dd00c9..326efa3 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
@@ -79,6 +79,7 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
+import java.lang.reflect.Constructor;
 import java.math.BigDecimal;
 import java.sql.Connection;
 import java.sql.DatabaseMetaData;
@@ -163,6 +164,8 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
  */
 
 public abstract class BaseTest {
+    public static final String DRIVER_CLASS_NAME_ATTRIB = "phoenix.driver.class.name";
+    
     private static final Map<String,String> tableDDLMap;
     private static final Logger logger = LoggerFactory.getLogger(BaseTest.class);
     protected static final int DEFAULT_TXN_TIMEOUT_SECONDS = 30;
@@ -440,7 +443,7 @@ public abstract class BaseTest {
      * @return url to be used by clients to connect to the cluster.
      * @throws IOException 
      */
-    protected static String setUpTestCluster(@Nonnull Configuration conf, ReadOnlyProps overrideProps) throws IOException {
+    protected static String setUpTestCluster(@Nonnull Configuration conf, ReadOnlyProps overrideProps) throws Exception {
         boolean isDistributedCluster = isDistributedClusterModeEnabled(conf);
         if (!isDistributedCluster) {
             return initMiniCluster(conf, overrideProps);
@@ -538,8 +541,9 @@ public abstract class BaseTest {
      * Initialize the mini cluster using phoenix-test specific configuration.
      * @param overrideProps TODO
      * @return url to be used by clients to connect to the mini cluster.
+     * @throws Exception 
      */
-    private static String initMiniCluster(Configuration conf, ReadOnlyProps overrideProps) {
+    private static String initMiniCluster(Configuration conf, ReadOnlyProps overrideProps) throws Exception {
         setUpConfigForMiniCluster(conf, overrideProps);
         utility = new HBaseTestingUtility(conf);
         try {
@@ -559,8 +563,9 @@ public abstract class BaseTest {
      * Initialize the cluster in distributed mode
      * @param overrideProps TODO
      * @return url to be used by clients to connect to the mini cluster.
+     * @throws Exception 
      */
-    private static String initClusterDistributedMode(Configuration conf, ReadOnlyProps overrideProps) {
+    private static String initClusterDistributedMode(Configuration conf, ReadOnlyProps overrideProps) throws Exception {
         setTestConfigForDistribuedCluster(conf, overrideProps);
         try {
             IntegrationTestingUtility util =  new IntegrationTestingUtility(conf);
@@ -572,13 +577,13 @@ public abstract class BaseTest {
         return JDBC_PROTOCOL + JDBC_PROTOCOL_TERMINATOR + PHOENIX_TEST_DRIVER_URL_PARAM;
     }
 
-    private static void setTestConfigForDistribuedCluster(Configuration conf, ReadOnlyProps overrideProps) {
+    private static void setTestConfigForDistribuedCluster(Configuration conf, ReadOnlyProps overrideProps) throws Exception {
         setDefaultTestConfig(conf, overrideProps);
     }
     
-    private static void setDefaultTestConfig(Configuration conf, ReadOnlyProps overrideProps) {
+    private static void setDefaultTestConfig(Configuration conf, ReadOnlyProps overrideProps) throws Exception {
         ConfigUtil.setReplicationConfigIfAbsent(conf);
-        QueryServices services = new PhoenixTestDriver().getQueryServices();
+        QueryServices services = newTestDriver(overrideProps).getQueryServices();
         for (Entry<String,String> entry : services.getProps()) {
             conf.set(entry.getKey(), entry.getValue());
         }
@@ -594,11 +599,11 @@ public abstract class BaseTest {
         }
     }
     
-    public static Configuration setUpConfigForMiniCluster(Configuration conf) {
+    public static Configuration setUpConfigForMiniCluster(Configuration conf) throws Exception {
         return setUpConfigForMiniCluster(conf, ReadOnlyProps.EMPTY_PROPS);
     }
     
-    public static Configuration setUpConfigForMiniCluster(Configuration conf, ReadOnlyProps overrideProps) {
+    public static Configuration setUpConfigForMiniCluster(Configuration conf, ReadOnlyProps overrideProps) throws Exception {
         assertNotNull(conf);
         setDefaultTestConfig(conf, overrideProps);
         /*
@@ -625,12 +630,24 @@ public abstract class BaseTest {
         return conf;
     }
 
+    private static PhoenixTestDriver newTestDriver(ReadOnlyProps props) throws Exception {
+        PhoenixTestDriver newDriver;
+        String driverClassName = props.get(DRIVER_CLASS_NAME_ATTRIB);
+        if (driverClassName == null) {
+            newDriver = new PhoenixTestDriver(props);
+        } else {
+            Class<?> clazz = Class.forName(driverClassName);
+            Constructor constr = clazz.getConstructor(ReadOnlyProps.class);
+            newDriver = (PhoenixTestDriver)constr.newInstance(props);
+        }
+        return newDriver;
+    }
     /**
      * Create a {@link PhoenixTestDriver} and register it.
      * @return an initialized and registered {@link PhoenixTestDriver} 
      */
     public static PhoenixTestDriver initAndRegisterTestDriver(String url, ReadOnlyProps props) throws Exception {
-        PhoenixTestDriver newDriver = new PhoenixTestDriver(props);
+        PhoenixTestDriver newDriver = newTestDriver(props);
         DriverManager.registerDriver(newDriver);
         Driver oldDriver = DriverManager.getDriver(url); 
         if (oldDriver != newDriver) {


[15/20] phoenix git commit: PHOENIX-4348 Point deletes do not work when there are immutable indexes with only row key columns

Posted by ja...@apache.org.
PHOENIX-4348 Point deletes do not work when there are immutable indexes with only row key columns


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

Branch: refs/heads/5.x-HBase-2.0
Commit: 8b4d6047070e1747d60a3e07e68b7c51fbc719aa
Parents: cdec814
Author: James Taylor <jt...@salesforce.com>
Authored: Thu Nov 2 18:47:01 2017 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Thu Nov 9 12:52:41 2017 -0800

----------------------------------------------------------------------
 .../org/apache/phoenix/end2end/DeleteIT.java    | 96 +++++++++++++++++++-
 .../apache/phoenix/compile/DeleteCompiler.java  |  5 +-
 2 files changed, 94 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/8b4d6047/phoenix-core/src/it/java/org/apache/phoenix/end2end/DeleteIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DeleteIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DeleteIT.java
index aa4d36e..9eac0af 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DeleteIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DeleteIT.java
@@ -18,6 +18,7 @@
 package org.apache.phoenix.end2end;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertTrue;
 
 import java.sql.Connection;
@@ -32,10 +33,7 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.util.QueryUtil;
-import org.apache.phoenix.util.TestUtil;
 import org.junit.Test;
 
 
@@ -339,8 +337,6 @@ public class DeleteIT extends ParallelStatsDisabledIT {
                 con.commit();
             }
             
-            TestUtil.dumpTable(con.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes(tableName)));
-            
             ResultSet rs = con.createStatement().executeQuery("SELECT /*+ NO_INDEX */ count(*) FROM " + tableName);
             assertTrue(rs.next());
             assertEquals(0, rs.getLong(1));
@@ -370,6 +366,96 @@ public class DeleteIT extends ParallelStatsDisabledIT {
         testDeleteRowFromTableWithImmutableIndex(true, false);
     }
     
+    @Test
+    public void testPointDeleteRowFromTableWithImmutableIndex() throws Exception {
+        testPointDeleteRowFromTableWithImmutableIndex(false, false);
+    }
+    
+    @Test
+    public void testPointDeleteRowFromTableWithLocalImmutableIndex() throws Exception {
+        testPointDeleteRowFromTableWithImmutableIndex(true, false);
+    }
+    
+    @Test
+    public void testPointDeleteRowFromTableWithImmutableIndex2() throws Exception {
+        testPointDeleteRowFromTableWithImmutableIndex(false, true);
+    }
+    
+    public void testPointDeleteRowFromTableWithImmutableIndex(boolean localIndex, boolean addNonPKIndex) throws Exception {
+        Connection con = null;
+        try {
+            boolean autoCommit = false;
+            con = DriverManager.getConnection(getUrl());
+            con.setAutoCommit(autoCommit);
+
+            Statement stm = con.createStatement();
+
+            String tableName = generateUniqueName();
+            String indexName1 = generateUniqueName();
+            String indexName2 = generateUniqueName();
+            String indexName3 = addNonPKIndex? generateUniqueName() : null;
+
+            stm.execute("CREATE TABLE IF NOT EXISTS " + tableName + " (" +
+                    "HOST CHAR(2) NOT NULL," +
+                    "DOMAIN VARCHAR NOT NULL, " +
+                    "FEATURE VARCHAR NOT NULL, " +
+                    "\"DATE\" DATE NOT NULL, \n" + 
+                    "USAGE.CORE BIGINT," +
+                    "USAGE.DB BIGINT," +
+                    "STATS.ACTIVE_VISITOR INTEGER " +
+                    "CONSTRAINT PK PRIMARY KEY (HOST, DOMAIN, FEATURE, \"DATE\")) IMMUTABLE_ROWS=true");
+            stm.execute("CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName1 + " ON " + tableName + " (\"DATE\", FEATURE)");
+            stm.execute("CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName2 + " ON " + tableName + " (FEATURE, DOMAIN)");
+            if (addNonPKIndex) {
+                stm.execute("CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName3 + " ON " + tableName + " (\"DATE\", FEATURE, USAGE.DB)");
+            }
+            
+            Date date = new Date(0);
+            PreparedStatement psInsert = con
+                    .prepareStatement("UPSERT INTO " + tableName + "(HOST, DOMAIN, FEATURE, \"DATE\", CORE, DB, ACTIVE_VISITOR) VALUES(?,?, ? , ?, ?, ?, ?)");
+            psInsert.setString(1, "AA");
+            psInsert.setString(2, "BB");
+            psInsert.setString(3, "CC");
+            psInsert.setDate(4, date);
+            psInsert.setLong(5, 1L);
+            psInsert.setLong(6, 2L);
+            psInsert.setLong(7, 3);
+            psInsert.execute();
+            if (!autoCommit) {
+                con.commit();
+            }
+            
+            String dml = "DELETE FROM " + tableName + " WHERE (HOST, DOMAIN, FEATURE, \"DATE\") = (?,?,?,?)";
+            PreparedStatement psDelete = con.prepareStatement(dml);
+            psDelete.setString(1, "AA");
+            psDelete.setString(2, "BB");
+            psDelete.setString(3, "CC");
+            psDelete.setDate(4, date);
+            psDelete.execute();
+            if (!autoCommit) {
+                con.commit();
+            }
+            psDelete = con.prepareStatement("EXPLAIN " + dml);
+            psDelete.setString(1, "AA");
+            psDelete.setString(2, "BB");
+            psDelete.setString(3, "CC");
+            psDelete.setDate(4, date);
+            String explainPlan = QueryUtil.getExplainPlan(psDelete.executeQuery());
+            if (addNonPKIndex) {
+                assertNotEquals("DELETE SINGLE ROW", explainPlan);
+            } else {
+                assertEquals("DELETE SINGLE ROW", explainPlan);
+            }
+            
+            assertDeleted(con, tableName, indexName1, indexName2, indexName3);
+        } finally {
+            try {
+                con.close();
+            } catch (Exception ex) {
+            }
+        }
+    }
+        
     public void testDeleteRowFromTableWithImmutableIndex(boolean localIndex, boolean useCoveredIndex) throws Exception {
         Connection con = null;
         try {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8b4d6047/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
index 73689d5..f038cda 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
@@ -331,8 +331,9 @@ public class DeleteCompiler {
         @Override
         public MutationState execute() throws SQLException {
             MutationState state = firstPlan.execute();
+            statement.getConnection().getMutationState().join(state);
             for (MutationPlan plan : plans.subList(1, plans.size())) {
-                plan.execute();
+                statement.getConnection().getMutationState().join(plan.execute());
             }
             return state;
         }
@@ -564,7 +565,7 @@ public class DeleteCompiler {
                         while (iterator.hasNext()) {
                             mutation.put(new ImmutableBytesPtr(iterator.next().getLowerRange()), new RowMutationState(PRow.DELETE_MARKER, statement.getConnection().getStatementExecutionCounter(), NULL_ROWTIMESTAMP_INFO, null));
                         }
-                        return new MutationState(context.getCurrentTable(), mutation, 0, maxSize, maxSizeBytes, connection);
+                        return new MutationState(plan.getTableRef(), mutation, 0, maxSize, maxSizeBytes, connection);
                     }
     
                     @Override


[10/20] phoenix git commit: PHOENIX-4343 In CREATE TABLE allow setting guide post width only on base data tables

Posted by ja...@apache.org.
PHOENIX-4343 In CREATE TABLE allow setting guide post width only on base data tables


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

Branch: refs/heads/5.x-HBase-2.0
Commit: e7d2c391ab208f2624b1b771452262506cce9730
Parents: fc327e7
Author: Samarth Jain <sa...@apache.org>
Authored: Wed Nov 1 23:21:01 2017 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Thu Nov 9 12:44:35 2017 -0800

----------------------------------------------------------------------
 .../apache/phoenix/end2end/CreateTableIT.java   | 73 ++++++++++++++++++++
 .../end2end/ExplainPlanWithStatsEnabledIT.java  |  2 +-
 .../phoenix/exception/SQLExceptionCode.java     |  2 +-
 .../apache/phoenix/schema/MetaDataClient.java   |  7 ++
 4 files changed, 82 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/e7d2c391/phoenix-core/src/it/java/org/apache/phoenix/end2end/CreateTableIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CreateTableIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CreateTableIT.java
index 93bb02b..1abc653 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CreateTableIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CreateTableIT.java
@@ -27,6 +27,7 @@ import static org.junit.Assert.fail;
 
 import java.sql.Connection;
 import java.sql.DriverManager;
+import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
@@ -35,6 +36,7 @@ import java.util.Properties;
 
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GlobalPermissionOrBuilder;
 import org.apache.hadoop.hbase.regionserver.BloomType;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.exception.SQLExceptionCode;
@@ -743,4 +745,75 @@ public class CreateTableIT extends ParallelStatsDisabledIT {
         }
         conn2.close();
     }
+
+    @Test
+    public void testSettingGuidePostWidth() throws Exception {
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String dataTable = generateUniqueName();
+            int guidePostWidth = 20;
+            String ddl =
+                    "CREATE TABLE " + dataTable + " (k INTEGER PRIMARY KEY, a bigint, b bigint)"
+                            + " GUIDE_POSTS_WIDTH=" + guidePostWidth;
+            conn.createStatement().execute(ddl);
+            assertEquals(20, checkGuidePostWidth(dataTable));
+            String viewName = "V_" + generateUniqueName();
+            ddl =
+                    "CREATE VIEW " + viewName + " AS SELECT * FROM " + dataTable
+                            + " GUIDE_POSTS_WIDTH=" + guidePostWidth;
+            try {
+                conn.createStatement().execute(ddl);
+            } catch (SQLException e) {
+                assertEquals(SQLExceptionCode.CANNOT_SET_GUIDE_POST_WIDTH.getErrorCode(),
+                    e.getErrorCode());
+            }
+
+            // let the view creation go through
+            ddl = "CREATE VIEW " + viewName + " AS SELECT * FROM " + dataTable;
+            conn.createStatement().execute(ddl);
+
+            String globalIndex = "GI_" + generateUniqueName();
+            ddl =
+                    "CREATE INDEX " + globalIndex + " ON " + dataTable
+                            + "(a) INCLUDE (b) GUIDE_POSTS_WIDTH = " + guidePostWidth;
+            try {
+                conn.createStatement().execute(ddl);
+            } catch (SQLException e) {
+                assertEquals(SQLExceptionCode.CANNOT_SET_GUIDE_POST_WIDTH.getErrorCode(),
+                    e.getErrorCode());
+            }
+            String localIndex = "LI_" + generateUniqueName();
+            ddl =
+                    "CREATE LOCAL INDEX " + localIndex + " ON " + dataTable
+                            + "(b) INCLUDE (a) GUIDE_POSTS_WIDTH = " + guidePostWidth;
+            try {
+                conn.createStatement().execute(ddl);
+            } catch (SQLException e) {
+                assertEquals(SQLExceptionCode.CANNOT_SET_GUIDE_POST_WIDTH.getErrorCode(),
+                    e.getErrorCode());
+            }
+            String viewIndex = "VI_" + generateUniqueName();
+            ddl =
+                    "CREATE LOCAL INDEX " + viewIndex + " ON " + dataTable
+                            + "(b) INCLUDE (a) GUIDE_POSTS_WIDTH = " + guidePostWidth;
+            try {
+                conn.createStatement().execute(ddl);
+            } catch (SQLException e) {
+                assertEquals(SQLExceptionCode.CANNOT_SET_GUIDE_POST_WIDTH.getErrorCode(),
+                    e.getErrorCode());
+            }
+        }
+    }
+
+    private int checkGuidePostWidth(String tableName) throws Exception {
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String query =
+                    "SELECT GUIDE_POSTS_WIDTH FROM SYSTEM.CATALOG WHERE TABLE_NAME = ? AND COLUMN_FAMILY IS NULL AND COLUMN_NAME IS NULL";
+            PreparedStatement stmt = conn.prepareStatement(query);
+            stmt.setString(1, tableName);
+            ResultSet rs = stmt.executeQuery();
+            assertTrue(rs.next());
+            return rs.getInt(1);
+        }
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e7d2c391/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
index b5e4588..e76b147 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
@@ -73,7 +73,7 @@ public class ExplainPlanWithStatsEnabledIT extends ParallelStatsEnabledIT {
             throws Exception {
         try (Connection conn = DriverManager.getConnection(getUrl())) {
             conn.createStatement().execute("CREATE INDEX " + indexName + " ON " + table
-                    + " (c1.a) INCLUDE (c2.b) GUIDE_POSTS_WIDTH = " + guidePostWidth);
+                    + " (c1.a) INCLUDE (c2.b) ");
             conn.createStatement().execute("UPDATE STATISTICS " + indexName);
         }
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e7d2c391/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java b/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
index cfeb212..e51fd9f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
@@ -378,7 +378,7 @@ public enum SQLExceptionCode {
     MAX_COLUMNS_EXCEEDED(1136, "XCL36", "The number of columns exceed the maximum supported by the table's qualifier encoding scheme"),
     INVALID_IMMUTABLE_STORAGE_SCHEME_AND_COLUMN_QUALIFIER_BYTES(1137, "XCL37", "If IMMUTABLE_STORAGE_SCHEME property is not set to ONE_CELL_PER_COLUMN COLUMN_ENCODED_BYTES cannot be 0"),
     INVALID_IMMUTABLE_STORAGE_SCHEME_CHANGE(1138, "XCL38", "IMMUTABLE_STORAGE_SCHEME property cannot be changed from/to ONE_CELL_PER_COLUMN "),
-
+    CANNOT_SET_GUIDE_POST_WIDTH(1139, "XCL39", "Guide post width can only be set on base data tables"),
     /**
      * Implementation defined class. Phoenix internal error. (errorcode 20, sqlstate INT).
      */

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e7d2c391/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 7ce2167..338b325 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
@@ -1949,6 +1949,13 @@ public class MetaDataClient {
             }
             String autoPartitionSeq = (String) TableProperty.AUTO_PARTITION_SEQ.getValue(tableProps);
             Long guidePostsWidth = (Long) TableProperty.GUIDE_POSTS_WIDTH.getValue(tableProps);
+
+            // We only allow setting guide post width for a base table
+            if (guidePostsWidth != null && tableType != PTableType.TABLE) {
+                throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_SET_GUIDE_POST_WIDTH)
+                        .setSchemaName(schemaName).setTableName(tableName).build().buildException();
+            }
+
             Boolean storeNullsProp = (Boolean) TableProperty.STORE_NULLS.getValue(tableProps);
             if (storeNullsProp == null) {
                 if (parent == null) {


[12/20] phoenix git commit: PHOENIX-4332 Indexes should inherit guide post width of the base data table

Posted by ja...@apache.org.
PHOENIX-4332 Indexes should inherit guide post width of the base data table


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

Branch: refs/heads/5.x-HBase-2.0
Commit: 59e49f78a311eadf6b3fbe7d30df20bd2f7e204c
Parents: e7d2c39
Author: Samarth Jain <sa...@apache.org>
Authored: Wed Nov 1 23:24:52 2017 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Thu Nov 9 12:50:59 2017 -0800

----------------------------------------------------------------------
 ...mnEncodedImmutableNonTxStatsCollectorIT.java |   1 +
 ...olumnEncodedImmutableTxStatsCollectorIT.java |   1 +
 ...lumnEncodedMutableNonTxStatsCollectorIT.java |   1 +
 .../ColumnEncodedMutableTxStatsCollectorIT.java |   1 +
 ...mnEncodedImmutableNonTxStatsCollectorIT.java |   1 +
 ...olumnEncodedImmutableTxStatsCollectorIT.java |   1 +
 .../phoenix/end2end/StatsCollectorIT.java       | 734 ----------------
 ...SysTableNamespaceMappedStatsCollectorIT.java |   1 +
 .../phoenix/schema/stats/StatsCollectorIT.java  | 831 +++++++++++++++++++
 .../stats/DefaultStatisticsCollector.java       |  58 +-
 10 files changed, 894 insertions(+), 736 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/59e49f78/phoenix-core/src/it/java/org/apache/phoenix/end2end/ColumnEncodedImmutableNonTxStatsCollectorIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ColumnEncodedImmutableNonTxStatsCollectorIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ColumnEncodedImmutableNonTxStatsCollectorIT.java
index d5d8442..eb01e89 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ColumnEncodedImmutableNonTxStatsCollectorIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ColumnEncodedImmutableNonTxStatsCollectorIT.java
@@ -20,6 +20,7 @@ package org.apache.phoenix.end2end;
 import java.util.Arrays;
 import java.util.Collection;
 
+import org.apache.phoenix.schema.stats.StatsCollectorIT;
 import org.junit.runners.Parameterized.Parameters;
 
 public class ColumnEncodedImmutableNonTxStatsCollectorIT extends StatsCollectorIT {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/59e49f78/phoenix-core/src/it/java/org/apache/phoenix/end2end/ColumnEncodedImmutableTxStatsCollectorIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ColumnEncodedImmutableTxStatsCollectorIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ColumnEncodedImmutableTxStatsCollectorIT.java
index 23b1654..4e90d70 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ColumnEncodedImmutableTxStatsCollectorIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ColumnEncodedImmutableTxStatsCollectorIT.java
@@ -20,6 +20,7 @@ package org.apache.phoenix.end2end;
 import java.util.Arrays;
 import java.util.Collection;
 
+import org.apache.phoenix.schema.stats.StatsCollectorIT;
 import org.junit.runners.Parameterized.Parameters;
 
 public class ColumnEncodedImmutableTxStatsCollectorIT extends StatsCollectorIT {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/59e49f78/phoenix-core/src/it/java/org/apache/phoenix/end2end/ColumnEncodedMutableNonTxStatsCollectorIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ColumnEncodedMutableNonTxStatsCollectorIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ColumnEncodedMutableNonTxStatsCollectorIT.java
index 24869a2..2a560db 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ColumnEncodedMutableNonTxStatsCollectorIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ColumnEncodedMutableNonTxStatsCollectorIT.java
@@ -20,6 +20,7 @@ package org.apache.phoenix.end2end;
 import java.util.Arrays;
 import java.util.Collection;
 
+import org.apache.phoenix.schema.stats.StatsCollectorIT;
 import org.junit.runners.Parameterized.Parameters;
 
 public class ColumnEncodedMutableNonTxStatsCollectorIT extends StatsCollectorIT {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/59e49f78/phoenix-core/src/it/java/org/apache/phoenix/end2end/ColumnEncodedMutableTxStatsCollectorIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ColumnEncodedMutableTxStatsCollectorIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ColumnEncodedMutableTxStatsCollectorIT.java
index eea591d..01fa2b5 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ColumnEncodedMutableTxStatsCollectorIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ColumnEncodedMutableTxStatsCollectorIT.java
@@ -20,6 +20,7 @@ package org.apache.phoenix.end2end;
 import java.util.Arrays;
 import java.util.Collection;
 
+import org.apache.phoenix.schema.stats.StatsCollectorIT;
 import org.junit.runners.Parameterized.Parameters;
 
 public class ColumnEncodedMutableTxStatsCollectorIT extends StatsCollectorIT {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/59e49f78/phoenix-core/src/it/java/org/apache/phoenix/end2end/NonColumnEncodedImmutableNonTxStatsCollectorIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/NonColumnEncodedImmutableNonTxStatsCollectorIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/NonColumnEncodedImmutableNonTxStatsCollectorIT.java
index fe70030..27c6dc2 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/NonColumnEncodedImmutableNonTxStatsCollectorIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/NonColumnEncodedImmutableNonTxStatsCollectorIT.java
@@ -20,6 +20,7 @@ package org.apache.phoenix.end2end;
 import java.util.Arrays;
 import java.util.Collection;
 
+import org.apache.phoenix.schema.stats.StatsCollectorIT;
 import org.junit.runners.Parameterized.Parameters;
 
 public class NonColumnEncodedImmutableNonTxStatsCollectorIT extends StatsCollectorIT {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/59e49f78/phoenix-core/src/it/java/org/apache/phoenix/end2end/NonColumnEncodedImmutableTxStatsCollectorIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/NonColumnEncodedImmutableTxStatsCollectorIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/NonColumnEncodedImmutableTxStatsCollectorIT.java
index 10a846a..0cec31a 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/NonColumnEncodedImmutableTxStatsCollectorIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/NonColumnEncodedImmutableTxStatsCollectorIT.java
@@ -20,6 +20,7 @@ package org.apache.phoenix.end2end;
 import java.util.Arrays;
 import java.util.Collection;
 
+import org.apache.phoenix.schema.stats.StatsCollectorIT;
 import org.junit.runners.Parameterized.Parameters;
 
 public class NonColumnEncodedImmutableTxStatsCollectorIT extends StatsCollectorIT {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/59e49f78/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java
deleted file mode 100644
index e18552a..0000000
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java
+++ /dev/null
@@ -1,734 +0,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.
- */
-package org.apache.phoenix.end2end;
-
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_STATS_TABLE;
-import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
-import static org.apache.phoenix.util.TestUtil.getAllSplits;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.sql.Array;
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.PreparedStatement;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Random;
-
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionLocation;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.phoenix.jdbc.PhoenixConnection;
-import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
-import org.apache.phoenix.query.ConnectionQueryServices;
-import org.apache.phoenix.query.KeyRange;
-import org.apache.phoenix.query.QueryServices;
-import org.apache.phoenix.query.QueryServicesOptions;
-import org.apache.phoenix.schema.PTable;
-import org.apache.phoenix.schema.PTableImpl;
-import org.apache.phoenix.schema.PTableKey;
-import org.apache.phoenix.schema.stats.GuidePostsInfo;
-import org.apache.phoenix.schema.stats.GuidePostsKey;
-import org.apache.phoenix.schema.stats.StatisticsUtil;
-import org.apache.phoenix.util.PropertiesUtil;
-import org.apache.phoenix.util.QueryUtil;
-import org.apache.phoenix.util.ReadOnlyProps;
-import org.apache.phoenix.util.SchemaUtil;
-import org.apache.phoenix.util.TestUtil;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Ignore;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import com.google.common.collect.Maps;
-
-@RunWith(Parameterized.class)
-public abstract class StatsCollectorIT extends BaseUniqueNamesOwnClusterIT {
-    private final String tableDDLOptions;
-    private final boolean columnEncoded;
-    private String tableName;
-    private String schemaName;
-    private String fullTableName;
-    private String physicalTableName;
-    private final boolean userTableNamespaceMapped;
-    private final boolean mutable;
-    
-    protected StatsCollectorIT(boolean mutable, boolean transactional, boolean userTableNamespaceMapped, boolean columnEncoded) {
-        StringBuilder sb = new StringBuilder();
-        if (transactional) {
-            sb.append("TRANSACTIONAL=true");
-        }
-        if (!columnEncoded) {
-            if (sb.length()>0) {
-                sb.append(",");
-            }
-            sb.append("COLUMN_ENCODED_BYTES=0");
-        } else {
-            if (sb.length()>0) {
-                sb.append(",");
-            }
-            sb.append("COLUMN_ENCODED_BYTES=4");
-        }
-        if (!mutable) {
-            if (sb.length()>0) {
-                sb.append(",");
-            }
-            sb.append("IMMUTABLE_ROWS=true");
-            if (!columnEncoded) {
-                sb.append(",IMMUTABLE_STORAGE_SCHEME="+PTableImpl.ImmutableStorageScheme.ONE_CELL_PER_COLUMN);
-            }
-        }
-        this.tableDDLOptions = sb.toString();
-        this.userTableNamespaceMapped = userTableNamespaceMapped;
-        this.columnEncoded = columnEncoded;
-        this.mutable = mutable;
-    }
-
-    @BeforeClass
-    public static void doSetup() throws Exception {
-        // enable name space mapping at global level on both client and server side
-        Map<String, String> serverProps = Maps.newHashMapWithExpectedSize(7);
-        serverProps.put(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, "true");
-        serverProps.put(QueryServices.STATS_GUIDEPOST_WIDTH_BYTES_ATTRIB, Long.toString(20));
-        Map<String, String> clientProps = Maps.newHashMapWithExpectedSize(2);
-        clientProps.put(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, "true");
-        clientProps.put(QueryServices.STATS_GUIDEPOST_WIDTH_BYTES_ATTRIB, Long.toString(20));
-        setUpTestDriver(new ReadOnlyProps(serverProps.entrySet().iterator()), new ReadOnlyProps(clientProps.entrySet().iterator()));
-    }
-    
-    @Before
-    public void generateTableNames() throws SQLException {
-        schemaName = generateUniqueName();
-        if (userTableNamespaceMapped) {
-            try (Connection conn = getConnection()) {
-                conn.createStatement().execute("CREATE SCHEMA " + schemaName);
-            }
-        }
-        tableName = "T_" + generateUniqueName();
-        fullTableName = SchemaUtil.getTableName(schemaName, tableName);
-        physicalTableName = SchemaUtil.getPhysicalHBaseTableName(schemaName, tableName, userTableNamespaceMapped).getString();
-    }
-
-    private Connection getConnection() throws SQLException {
-        return getConnection(Integer.MAX_VALUE);
-    }
-
-    private Connection getConnection(Integer statsUpdateFreq) throws SQLException {
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        props.setProperty(QueryServices.EXPLAIN_CHUNK_COUNT_ATTRIB, Boolean.TRUE.toString());
-        props.setProperty(QueryServices.EXPLAIN_ROW_COUNT_ATTRIB, Boolean.TRUE.toString());
-        props.setProperty(QueryServices.STATS_UPDATE_FREQ_MS_ATTRIB, Integer.toString(statsUpdateFreq));
-        // enable/disable namespace mapping at connection level
-        props.setProperty(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.toString(userTableNamespaceMapped));
-        return DriverManager.getConnection(getUrl(), props);
-    }
-    
-    @Test
-    public void testUpdateEmptyStats() throws Exception {
-        Connection conn = getConnection();
-        conn.setAutoCommit(true);
-        conn.createStatement().execute(
-                "CREATE TABLE " + fullTableName +" ( k CHAR(1) PRIMARY KEY )"  + tableDDLOptions);
-        conn.createStatement().execute("UPDATE STATISTICS " + fullTableName);
-        ResultSet rs = conn.createStatement().executeQuery("EXPLAIN SELECT * FROM " + fullTableName);
-        String explainPlan = QueryUtil.getExplainPlan(rs);
-        assertEquals(
-                "CLIENT 1-CHUNK 0 ROWS 20 BYTES PARALLEL 1-WAY FULL SCAN OVER " + physicalTableName + "\n" +
-                "    SERVER FILTER BY FIRST KEY ONLY",
-                explainPlan);
-        conn.close();
-    }
-    
-    @Test
-    public void testSomeUpdateEmptyStats() throws Exception {
-        Connection conn = getConnection();
-        conn.setAutoCommit(true);
-        conn.createStatement().execute(
-                "CREATE TABLE " + fullTableName +" ( k VARCHAR PRIMARY KEY, a.v1 VARCHAR, b.v2 VARCHAR ) " + tableDDLOptions + (tableDDLOptions.isEmpty() ? "" : ",") + "SALT_BUCKETS = 3");
-        conn.createStatement().execute("UPSERT INTO " + fullTableName + "(k,v1) VALUES('a','123456789')");
-        conn.createStatement().execute("UPDATE STATISTICS " + fullTableName);
-                
-        ResultSet rs;
-        String explainPlan;
-        rs = conn.createStatement().executeQuery("EXPLAIN SELECT v2 FROM " + fullTableName + " WHERE v2='foo'");
-        explainPlan = QueryUtil.getExplainPlan(rs);
-        // if we are using the ONE_CELL_PER_COLUMN_FAMILY storage scheme, we will have the single kv even though there are no values for col family v2 
-        String stats = columnEncoded && !mutable  ? "4-CHUNK 1 ROWS 38 BYTES" : "3-CHUNK 0 ROWS 20 BYTES";
-        assertEquals(
-                "CLIENT " + stats + " PARALLEL 3-WAY FULL SCAN OVER " + physicalTableName + "\n" +
-                "    SERVER FILTER BY B.V2 = 'foo'\n" + 
-                "CLIENT MERGE SORT",
-                explainPlan);
-        rs = conn.createStatement().executeQuery("EXPLAIN SELECT * FROM " + fullTableName);
-        explainPlan = QueryUtil.getExplainPlan(rs);
-        assertEquals(
-                "CLIENT 4-CHUNK 1 ROWS " + (columnEncoded ? "28" : "34") + " BYTES PARALLEL 3-WAY FULL SCAN OVER " + physicalTableName + "\n" +
-                "CLIENT MERGE SORT",
-                explainPlan);
-        rs = conn.createStatement().executeQuery("EXPLAIN SELECT * FROM " + fullTableName + " WHERE k = 'a'");
-        explainPlan = QueryUtil.getExplainPlan(rs);
-        assertEquals(
-                "CLIENT 1-CHUNK 1 ROWS " + (columnEncoded ? "204" : "202") + " BYTES PARALLEL 1-WAY POINT LOOKUP ON 1 KEY OVER " + physicalTableName + "\n" +
-                "CLIENT MERGE SORT",
-                explainPlan);
-        
-        conn.close();
-    }
-    
-    @Test
-    public void testUpdateStats() throws SQLException, IOException,
-			InterruptedException {
-		Connection conn;
-        PreparedStatement stmt;
-        ResultSet rs;
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        conn = getConnection();
-        conn.createStatement().execute(
-                "CREATE TABLE " + fullTableName +" ( k VARCHAR, a_string_array VARCHAR(100) ARRAY[4], b_string_array VARCHAR(100) ARRAY[4] \n"
-                        + " CONSTRAINT pk PRIMARY KEY (k, b_string_array DESC))"
-                		+ tableDDLOptions );
-        String[] s;
-        Array array;
-        conn = upsertValues(props, fullTableName);
-        // CAll the update statistics query here. If already major compaction has run this will not get executed.
-        stmt = conn.prepareStatement("UPDATE STATISTICS " + fullTableName);
-        stmt.execute();
-        stmt = upsertStmt(conn, fullTableName);
-        stmt.setString(1, "z");
-        s = new String[] { "xyz", "def", "ghi", "jkll", null, null, "xxx" };
-        array = conn.createArrayOf("VARCHAR", s);
-        stmt.setArray(2, array);
-        s = new String[] { "zya", "def", "ghi", "jkll", null, null, null, "xxx" };
-        array = conn.createArrayOf("VARCHAR", s);
-        stmt.setArray(3, array);
-        stmt.execute();
-        stmt = conn.prepareStatement("UPDATE STATISTICS " + fullTableName);
-        stmt.execute();
-        rs = conn.createStatement().executeQuery("SELECT k FROM " + fullTableName);
-        assertTrue(rs.next());
-        conn.close();
-    }
-
-    private void testNoDuplicatesAfterUpdateStats(String splitKey) throws Throwable {
-        Connection conn = getConnection();
-        PreparedStatement stmt;
-        ResultSet rs;
-        conn.createStatement()
-                .execute("CREATE TABLE " + fullTableName
-                        + " ( k VARCHAR, c1.a bigint,c2.b bigint CONSTRAINT pk PRIMARY KEY (k))"+ tableDDLOptions
-                        + (splitKey != null ? " split on (" + splitKey + ")" : "") );
-        conn.createStatement().execute("upsert into " + fullTableName + " values ('abc',1,3)");
-        conn.createStatement().execute("upsert into " + fullTableName + " values ('def',2,4)");
-        conn.commit();
-        conn.createStatement().execute("UPDATE STATISTICS " + fullTableName);
-        rs = conn.createStatement().executeQuery("SELECT k FROM " + fullTableName + " order by k desc");
-        assertTrue(rs.next());
-        assertEquals("def", rs.getString(1));
-        assertTrue(rs.next());
-        assertEquals("abc", rs.getString(1));
-        assertTrue(!rs.next());
-        conn.close();
-    }
-
-    @Test
-    public void testNoDuplicatesAfterUpdateStatsWithSplits() throws Throwable {
-        testNoDuplicatesAfterUpdateStats("'abc','def'");
-    }
-
-    @Test
-    public void testNoDuplicatesAfterUpdateStatsWithDesc() throws Throwable {
-        testNoDuplicatesAfterUpdateStats(null);
-    }
-
-    @Test
-    public void testUpdateStatsWithMultipleTables() throws Throwable {
-        String fullTableName2 = SchemaUtil.getTableName(schemaName, "T_" + generateUniqueName());
-        Connection conn;
-        PreparedStatement stmt;
-        ResultSet rs;
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        conn = getConnection();
-        conn.createStatement().execute(
-                "CREATE TABLE " + fullTableName +" ( k VARCHAR, a_string_array VARCHAR(100) ARRAY[4], b_string_array VARCHAR(100) ARRAY[4] \n"
-                        + " CONSTRAINT pk PRIMARY KEY (k, b_string_array DESC))" + tableDDLOptions );
-        conn.createStatement().execute(
-                "CREATE TABLE " + fullTableName2 +" ( k VARCHAR, a_string_array VARCHAR(100) ARRAY[4], b_string_array VARCHAR(100) ARRAY[4] \n"
-                        + " CONSTRAINT pk PRIMARY KEY (k, b_string_array DESC))" + tableDDLOptions );
-        String[] s;
-        Array array;
-        conn = upsertValues(props, fullTableName);
-        conn = upsertValues(props, fullTableName2);
-        // CAll the update statistics query here
-        stmt = conn.prepareStatement("UPDATE STATISTICS "+fullTableName);
-        stmt.execute();
-        stmt = conn.prepareStatement("UPDATE STATISTICS "+fullTableName2);
-        stmt.execute();
-        stmt = upsertStmt(conn, fullTableName);
-        stmt.setString(1, "z");
-        s = new String[] { "xyz", "def", "ghi", "jkll", null, null, "xxx" };
-        array = conn.createArrayOf("VARCHAR", s);
-        stmt.setArray(2, array);
-        s = new String[] { "zya", "def", "ghi", "jkll", null, null, null, "xxx" };
-        array = conn.createArrayOf("VARCHAR", s);
-        stmt.setArray(3, array);
-        stmt.execute();
-        stmt = upsertStmt(conn, fullTableName2);
-        stmt.setString(1, "z");
-        s = new String[] { "xyz", "def", "ghi", "jkll", null, null, "xxx" };
-        array = conn.createArrayOf("VARCHAR", s);
-        stmt.setArray(2, array);
-        s = new String[] { "zya", "def", "ghi", "jkll", null, null, null, "xxx" };
-        array = conn.createArrayOf("VARCHAR", s);
-        stmt.setArray(3, array);
-        stmt.execute();
-        conn.close();
-        conn = getConnection();
-        // This analyze would not work
-        stmt = conn.prepareStatement("UPDATE STATISTICS "+fullTableName2);
-        stmt.execute();
-        rs = conn.createStatement().executeQuery("SELECT k FROM "+fullTableName2);
-        assertTrue(rs.next());
-        conn.close();
-    }
-
-    private Connection upsertValues(Properties props, String tableName) throws SQLException, IOException,
-            InterruptedException {
-        Connection conn;
-        PreparedStatement stmt;
-        conn = getConnection();
-        stmt = upsertStmt(conn, tableName);
-        stmt.setString(1, "a");
-        String[] s = new String[] { "abc", "def", "ghi", "jkll", null, null, "xxx" };
-        Array array = conn.createArrayOf("VARCHAR", s);
-        stmt.setArray(2, array);
-        s = new String[] { "abc", "def", "ghi", "jkll", null, null, null, "xxx" };
-        array = conn.createArrayOf("VARCHAR", s);
-        stmt.setArray(3, array);
-        stmt.execute();
-        conn.commit();
-        stmt = upsertStmt(conn, tableName);
-        stmt.setString(1, "b");
-        s = new String[] { "xyz", "def", "ghi", "jkll", null, null, "xxx" };
-        array = conn.createArrayOf("VARCHAR", s);
-        stmt.setArray(2, array);
-        s = new String[] { "zya", "def", "ghi", "jkll", null, null, null, "xxx" };
-        array = conn.createArrayOf("VARCHAR", s);
-        stmt.setArray(3, array);
-        stmt.execute();
-        conn.commit();
-        stmt = upsertStmt(conn, tableName);
-        stmt.setString(1, "c");
-        s = new String[] { "xyz", "def", "ghi", "jkll", null, null, "xxx" };
-        array = conn.createArrayOf("VARCHAR", s);
-        stmt.setArray(2, array);
-        s = new String[] { "zya", "def", "ghi", "jkll", null, null, null, "xxx" };
-        array = conn.createArrayOf("VARCHAR", s);
-        stmt.setArray(3, array);
-        stmt.execute();
-        conn.commit();
-        stmt = upsertStmt(conn, tableName);
-        stmt.setString(1, "d");
-        s = new String[] { "xyz", "def", "ghi", "jkll", null, null, "xxx" };
-        array = conn.createArrayOf("VARCHAR", s);
-        stmt.setArray(2, array);
-        s = new String[] { "zya", "def", "ghi", "jkll", null, null, null, "xxx" };
-        array = conn.createArrayOf("VARCHAR", s);
-        stmt.setArray(3, array);
-        stmt.execute();
-        conn.commit();
-        stmt = upsertStmt(conn, tableName);
-        stmt.setString(1, "b");
-        s = new String[] { "xyz", "def", "ghi", "jkll", null, null, "xxx" };
-        array = conn.createArrayOf("VARCHAR", s);
-        stmt.setArray(2, array);
-        s = new String[] { "zya", "def", "ghi", "jkll", null, null, null, "xxx" };
-        array = conn.createArrayOf("VARCHAR", s);
-        stmt.setArray(3, array);
-        stmt.execute();
-        conn.commit();
-        stmt = upsertStmt(conn, tableName);
-        stmt.setString(1, "e");
-        s = new String[] { "xyz", "def", "ghi", "jkll", null, null, "xxx" };
-        array = conn.createArrayOf("VARCHAR", s);
-        stmt.setArray(2, array);
-        s = new String[] { "zya", "def", "ghi", "jkll", null, null, null, "xxx" };
-        array = conn.createArrayOf("VARCHAR", s);
-        stmt.setArray(3, array);
-        stmt.execute();
-        conn.commit();
-        return conn;
-    }
-
-    private PreparedStatement upsertStmt(Connection conn, String tableName) throws SQLException {
-        PreparedStatement stmt;
-        stmt = conn.prepareStatement("UPSERT INTO " + tableName + " VALUES(?,?,?)");
-        return stmt;
-    }
-
-    private void compactTable(Connection conn, String tableName) throws Exception {
-        TestUtil.doMajorCompaction(conn, tableName);
-    }
-    
-    @Test
-    @Ignore //TODO remove this once  https://issues.apache.org/jira/browse/TEPHRA-208 is fixed
-    public void testCompactUpdatesStats() throws Exception {
-        testCompactUpdatesStats(0, fullTableName);
-    }
-    
-    @Test
-    @Ignore //TODO remove this once  https://issues.apache.org/jira/browse/TEPHRA-208 is fixed
-    public void testCompactUpdatesStatsWithMinStatsUpdateFreq() throws Exception {
-        testCompactUpdatesStats(QueryServicesOptions.DEFAULT_STATS_UPDATE_FREQ_MS, fullTableName);
-    }
-    
-    private static void invalidateStats(Connection conn, String tableName) throws SQLException {
-        PTable ptable = conn.unwrap(PhoenixConnection.class)
-                .getMetaDataCache().getTableRef(new PTableKey(null, tableName))
-                .getTable();
-        byte[] name = ptable.getPhysicalName().getBytes();
-        conn.unwrap(PhoenixConnection.class).getQueryServices().invalidateStats(new GuidePostsKey(name, SchemaUtil.getEmptyColumnFamily(ptable)));
-    }
-    
-    private void testCompactUpdatesStats(Integer statsUpdateFreq, String tableName) throws Exception {
-        int nRows = 10;
-        Connection conn = getConnection(statsUpdateFreq);
-        PreparedStatement stmt;
-        conn.createStatement().execute("CREATE TABLE " + tableName + "(k CHAR(1) PRIMARY KEY, v INTEGER, w INTEGER) "
-                + (!tableDDLOptions.isEmpty() ? tableDDLOptions + "," : "") 
-                + HColumnDescriptor.KEEP_DELETED_CELLS + "=" + Boolean.FALSE);
-        stmt = conn.prepareStatement("UPSERT INTO " + tableName + " VALUES(?,?,?)");
-        for (int i = 0; i < nRows; i++) {
-            stmt.setString(1, Character.toString((char) ('a' + i)));
-            stmt.setInt(2, i);
-            stmt.setInt(3, i);
-            stmt.executeUpdate();
-        }
-        conn.commit();
-        
-        compactTable(conn, physicalTableName);
-        
-        if (statsUpdateFreq != 0) {
-            invalidateStats(conn, tableName);
-        } else {
-            // Confirm that when we have a non zero STATS_UPDATE_FREQ_MS_ATTRIB, after we run
-            // UPDATATE STATISTICS, the new statistics are faulted in as expected.
-            List<KeyRange>keyRanges = getAllSplits(conn, tableName);
-            assertNotEquals(nRows+1, keyRanges.size());
-            // If we've set MIN_STATS_UPDATE_FREQ_MS_ATTRIB, an UPDATE STATISTICS will invalidate the cache
-            // and forcing the new stats to be pulled over.
-            int rowCount = conn.createStatement().executeUpdate("UPDATE STATISTICS " + tableName);
-            assertEquals(10, rowCount);
-        }
-        List<KeyRange>keyRanges = getAllSplits(conn, tableName);
-        assertEquals(nRows+1, keyRanges.size());
-        
-        int nDeletedRows = conn.createStatement().executeUpdate("DELETE FROM " + tableName + " WHERE V < " + nRows / 2);
-        conn.commit();
-        assertEquals(5, nDeletedRows);
-        
-        Scan scan = new Scan();
-        scan.setRaw(true);
-        PhoenixConnection phxConn = conn.unwrap(PhoenixConnection.class);
-        try (Table htable = phxConn.getQueryServices().getTable(Bytes.toBytes(tableName))) {
-            ResultScanner scanner = htable.getScanner(scan);
-            Result result;
-            while ((result = scanner.next())!=null) {
-                System.out.println(result);
-            }
-        }
-        
-        compactTable(conn, physicalTableName);
-        
-        scan = new Scan();
-        scan.setRaw(true);
-        phxConn = conn.unwrap(PhoenixConnection.class);
-        try (Table htable = phxConn.getQueryServices().getTable(Bytes.toBytes(tableName))) {
-            ResultScanner scanner = htable.getScanner(scan);
-            Result result;
-            while ((result = scanner.next())!=null) {
-                System.out.println(result);
-            }
-        }
-        
-        if (statsUpdateFreq != 0) {
-            invalidateStats(conn, tableName);
-        } else {
-            assertEquals(nRows+1, keyRanges.size());
-            // If we've set STATS_UPDATE_FREQ_MS_ATTRIB, an UPDATE STATISTICS will invalidate the cache
-            // and force us to pull over the new stats
-            int rowCount = conn.createStatement().executeUpdate("UPDATE STATISTICS " + tableName);
-            assertEquals(5, rowCount);
-        }
-        keyRanges = getAllSplits(conn, tableName);
-        assertEquals(nRows/2+1, keyRanges.size());
-        ResultSet rs = conn.createStatement().executeQuery("SELECT SUM(GUIDE_POSTS_ROW_COUNT) FROM "
-                + "\""+ SYSTEM_CATALOG_SCHEMA + "\".\"" + SYSTEM_STATS_TABLE + "\"" + " WHERE PHYSICAL_NAME='" + physicalTableName + "'");
-        rs.next();
-        assertEquals(nRows - nDeletedRows, rs.getLong(1));
-    }
-
-    @Test
-    public void testWithMultiCF() throws Exception {
-        int nRows = 20;
-        Connection conn = getConnection(0);
-        PreparedStatement stmt;
-        conn.createStatement().execute(
-                "CREATE TABLE " + fullTableName
-                        + "(k VARCHAR PRIMARY KEY, a.v INTEGER, b.v INTEGER, c.v INTEGER NULL, d.v INTEGER NULL) "
-                        + tableDDLOptions );
-        stmt = conn.prepareStatement("UPSERT INTO " + fullTableName + " VALUES(?,?, ?, ?, ?)");
-        byte[] val = new byte[250];
-        for (int i = 0; i < nRows; i++) {
-            stmt.setString(1, Character.toString((char)('a' + i)) + Bytes.toString(val));
-            stmt.setInt(2, i);
-            stmt.setInt(3, i);
-            stmt.setInt(4, i);
-            stmt.setInt(5, i);
-            stmt.executeUpdate();
-        }
-        conn.commit();
-        stmt = conn.prepareStatement("UPSERT INTO " + fullTableName + "(k, c.v, d.v) VALUES(?,?,?)");
-        for (int i = 0; i < 5; i++) {
-            stmt.setString(1, Character.toString((char)('a' + 'z' + i)) + Bytes.toString(val));
-            stmt.setInt(2, i);
-            stmt.setInt(3, i);
-            stmt.executeUpdate();
-        }
-        conn.commit();
-
-        ResultSet rs;
-        TestUtil.analyzeTable(conn, fullTableName);
-        List<KeyRange> keyRanges = getAllSplits(conn, fullTableName);
-        assertEquals(26, keyRanges.size());
-        rs = conn.createStatement().executeQuery("EXPLAIN SELECT * FROM " + fullTableName);
-        assertEquals("CLIENT 26-CHUNK 25 ROWS " + (columnEncoded ? ( mutable ? "12530" : "13902" ) : "12420") + " BYTES PARALLEL 1-WAY FULL SCAN OVER " + physicalTableName,
-                QueryUtil.getExplainPlan(rs));
-
-        ConnectionQueryServices services = conn.unwrap(PhoenixConnection.class).getQueryServices();
-        List<HRegionLocation> regions = services.getAllTableRegions(Bytes.toBytes(physicalTableName));
-        assertEquals(1, regions.size());
-
-        TestUtil.analyzeTable(conn, fullTableName);
-        String query = "UPDATE STATISTICS " + fullTableName + " SET \""
-                + QueryServices.STATS_GUIDEPOST_WIDTH_BYTES_ATTRIB + "\"=" + Long.toString(1000);
-        conn.createStatement().execute(query);
-        keyRanges = getAllSplits(conn, fullTableName);
-        boolean oneCellPerColFamliyStorageScheme = !mutable && columnEncoded;
-        assertEquals(oneCellPerColFamliyStorageScheme ? 13 : 12, keyRanges.size());
-
-        rs = conn
-                .createStatement()
-                .executeQuery(
-                        "SELECT COLUMN_FAMILY,SUM(GUIDE_POSTS_ROW_COUNT),SUM(GUIDE_POSTS_WIDTH),COUNT(*) from \"SYSTEM\".STATS where PHYSICAL_NAME = '"
-                                + physicalTableName + "' GROUP BY COLUMN_FAMILY ORDER BY COLUMN_FAMILY");
-
-        assertTrue(rs.next());
-        assertEquals("A", rs.getString(1));
-        assertEquals(24, rs.getInt(2));
-        assertEquals(columnEncoded ? ( mutable ? 12252 : 13624 ) : 12144, rs.getInt(3));
-        assertEquals(oneCellPerColFamliyStorageScheme ? 12 : 11, rs.getInt(4));
-
-        assertTrue(rs.next());
-        assertEquals("B", rs.getString(1));
-        assertEquals(oneCellPerColFamliyStorageScheme ? 24 : 20, rs.getInt(2));
-        assertEquals(columnEncoded ? ( mutable ? 5600 : 6972 ) : 5540, rs.getInt(3));
-        assertEquals(oneCellPerColFamliyStorageScheme ? 6 : 5, rs.getInt(4));
-
-        assertTrue(rs.next());
-        assertEquals("C", rs.getString(1));
-        assertEquals(24, rs.getInt(2));
-        assertEquals(columnEncoded ? ( mutable ? 6724 : 6988 ) : 6652, rs.getInt(3));
-        assertEquals(6, rs.getInt(4));
-
-        assertTrue(rs.next());
-        assertEquals("D", rs.getString(1));
-        assertEquals(24, rs.getInt(2));
-        assertEquals(columnEncoded ? ( mutable ? 6724 : 6988 ) : 6652, rs.getInt(3));
-        assertEquals(6, rs.getInt(4));
-
-        assertFalse(rs.next());
-        
-        // Disable stats
-        conn.createStatement().execute("ALTER TABLE " + fullTableName + 
-                " SET " + PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH + "=0");
-        TestUtil.analyzeTable(conn, fullTableName);
-        // Assert that there are no more guideposts
-        rs = conn.createStatement().executeQuery("SELECT count(1) FROM " + PhoenixDatabaseMetaData.SYSTEM_STATS_NAME + 
-                " WHERE " + PhoenixDatabaseMetaData.PHYSICAL_NAME + "='" + physicalTableName + "' AND " + PhoenixDatabaseMetaData.COLUMN_FAMILY + " IS NOT NULL");
-        assertTrue(rs.next());
-        assertEquals(0, rs.getLong(1));
-        assertFalse(rs.next());
-        rs = conn.createStatement().executeQuery("EXPLAIN SELECT * FROM " + fullTableName);
-        assertEquals("CLIENT 1-CHUNK PARALLEL 1-WAY FULL SCAN OVER " + physicalTableName,
-                QueryUtil.getExplainPlan(rs));
-    }
-
-    @Test
-    public void testRowCountAndByteCounts() throws SQLException {
-        Connection conn = getConnection();
-        String ddl = "CREATE TABLE " + fullTableName + " (t_id VARCHAR NOT NULL,\n" + "k1 INTEGER NOT NULL,\n"
-                + "k2 INTEGER NOT NULL,\n" + "C3.k3 INTEGER,\n" + "C2.v1 VARCHAR,\n"
-                + "CONSTRAINT pk PRIMARY KEY (t_id, k1, k2)) " + tableDDLOptions + " split on ('e','j','o')";
-        conn.createStatement().execute(ddl);
-        String[] strings = { "a", "b", "c", "d", "e", "f", "g", "h", "i", "j", "k", "l", "m", "n", "o", "p", "q", "r",
-                "s", "t", "u", "v", "w", "x", "y", "z" };
-        for (int i = 0; i < 26; i++) {
-            conn.createStatement().execute(
-                    "UPSERT INTO " + fullTableName + " values('" + strings[i] + "'," + i + "," + (i + 1) + ","
-                            + (i + 2) + ",'" + strings[25 - i] + "')");
-        }
-        conn.commit();
-        ResultSet rs;
-        String query = "UPDATE STATISTICS " + fullTableName + " SET \""
-                + QueryServices.STATS_GUIDEPOST_WIDTH_BYTES_ATTRIB + "\"=" + Long.toString(20);
-        conn.createStatement().execute(query);
-        Random r = new Random();
-        int count = 0;
-        while (count < 4) {
-            int startIndex = r.nextInt(strings.length);
-            int endIndex = r.nextInt(strings.length - startIndex) + startIndex;
-            long rows = endIndex - startIndex;
-            long c2Bytes = rows * (columnEncoded ? ( mutable ? 37 : 48 ) : 35);
-            String physicalTableName = SchemaUtil.getPhysicalTableName(Bytes.toBytes(fullTableName), userTableNamespaceMapped).toString();
-            rs = conn.createStatement().executeQuery(
-                    "SELECT COLUMN_FAMILY,SUM(GUIDE_POSTS_ROW_COUNT),SUM(GUIDE_POSTS_WIDTH) from \"SYSTEM\".STATS where PHYSICAL_NAME = '"
-                            + physicalTableName + "' AND GUIDE_POST_KEY>= cast('" + strings[startIndex]
-                            + "' as varbinary) AND  GUIDE_POST_KEY<cast('" + strings[endIndex]
-                            + "' as varbinary) and COLUMN_FAMILY='C2' group by COLUMN_FAMILY");
-            if (startIndex < endIndex) {
-                assertTrue(rs.next());
-                assertEquals("C2", rs.getString(1));
-                assertEquals(rows, rs.getLong(2));
-                assertEquals(c2Bytes, rs.getLong(3));
-                count++;
-            }
-        }
-    }
-
-    @Test
-    public void testRowCountWhenNumKVsExceedCompactionScannerThreshold() throws Exception {
-        String tableName = generateUniqueName();
-        StringBuilder sb = new StringBuilder(200);
-        sb.append("CREATE TABLE " + tableName + "(PK1 VARCHAR NOT NULL, ");
-        int numRows = 10;
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
-            int compactionScannerKVThreshold =
-                    conn.unwrap(PhoenixConnection.class).getQueryServices().getConfiguration()
-                            .getInt(HConstants.COMPACTION_KV_MAX,
-                                HConstants.COMPACTION_KV_MAX_DEFAULT);
-            int numKvColumns = compactionScannerKVThreshold * 2;
-            for (int i = 1; i <= numKvColumns; i++) {
-                sb.append("KV" + i + " VARCHAR");
-                if (i < numKvColumns) {
-                    sb.append(", ");
-                }
-            }
-            sb.append(" CONSTRAINT PK PRIMARY KEY (PK1))");
-            String ddl = sb.toString();
-            conn.createStatement().execute(ddl);
-            sb = new StringBuilder(200);
-            sb.append("UPSERT INTO " + tableName + " VALUES (");
-            for (int i = 1; i <= numKvColumns + 1; i++) {
-                sb.append("?");
-                if (i < numKvColumns + 1) {
-                    sb.append(", ");
-                }
-            }
-            sb.append(")");
-            String dml = sb.toString();
-            PreparedStatement stmt = conn.prepareStatement(dml);
-            String keyValue = "KVVVVVV";
-            for (int j = 1; j <= numRows; j++) {
-                for (int i = 1; i <= numKvColumns + 1; i++) {
-                    if (i == 1) {
-                        stmt.setString(1, "" + j);
-                    } else {
-                        stmt.setString(i, keyValue);
-                    }
-                }
-                stmt.executeUpdate();
-            }
-            conn.commit();
-            conn.createStatement().execute("UPDATE STATISTICS " + tableName);
-            String q = "SELECT SUM(GUIDE_POSTS_ROW_COUNT) FROM SYSTEM.STATS WHERE PHYSICAL_NAME = '" + tableName + "'";
-            ResultSet rs = conn.createStatement().executeQuery(q);
-            rs.next();
-            assertEquals("Number of expected rows in stats table after update stats didn't match!", numRows, rs.getInt(1));
-            conn.createStatement().executeUpdate("DELETE FROM SYSTEM.STATS WHERE PHYSICAL_NAME = '" + tableName + "'");
-            conn.commit();
-            TestUtil.doMajorCompaction(conn, tableName);
-            q = "SELECT SUM(GUIDE_POSTS_ROW_COUNT) FROM SYSTEM.STATS WHERE PHYSICAL_NAME = '" + tableName + "'";
-            rs = conn.createStatement().executeQuery(q);
-            rs.next();
-            assertEquals("Number of expected rows in stats table after major compaction didn't match", numRows, rs.getInt(1));
-        }
-    }
-    
-    @Test
-    public void testEmptyGuidePostGeneratedWhenDataSizeLessThanGPWidth() throws Exception {
-        String tableName = generateUniqueName();
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
-            long guidePostWidth = 20000000;
-            conn.createStatement()
-                    .execute("CREATE TABLE " + tableName
-                            + " ( k INTEGER, c1.a bigint,c2.b bigint CONSTRAINT pk PRIMARY KEY (k)) GUIDE_POSTS_WIDTH="
-                            + guidePostWidth + ", SALT_BUCKETS = 4");
-            conn.createStatement().execute("upsert into " + tableName + " values (100,1,3)");
-            conn.createStatement().execute("upsert into " + tableName + " values (101,2,4)");
-            conn.commit();
-            conn.createStatement().execute("UPDATE STATISTICS " + tableName);
-            ConnectionQueryServices queryServices =
-                    conn.unwrap(PhoenixConnection.class).getQueryServices();
-            try (Table statsHTable =
-                    queryServices.getTable(
-                        SchemaUtil.getPhysicalName(PhoenixDatabaseMetaData.SYSTEM_STATS_NAME_BYTES,
-                            queryServices.getProps()).getName())) {
-                GuidePostsInfo gps =
-                        StatisticsUtil.readStatistics(statsHTable,
-                            new GuidePostsKey(Bytes.toBytes(tableName), Bytes.toBytes("C1")),
-                            HConstants.LATEST_TIMESTAMP);
-                assertTrue(gps.isEmptyGuidePost());
-                assertEquals(guidePostWidth, gps.getByteCounts()[0]);
-                assertTrue(gps.getGuidePostTimestamps()[0] > 0);
-                gps =
-                        StatisticsUtil.readStatistics(statsHTable,
-                            new GuidePostsKey(Bytes.toBytes(tableName), Bytes.toBytes("C2")),
-                            HConstants.LATEST_TIMESTAMP);
-                assertTrue(gps.isEmptyGuidePost());
-                assertEquals(guidePostWidth, gps.getByteCounts()[0]);
-                assertTrue(gps.getGuidePostTimestamps()[0] > 0);
-            }
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/59e49f78/phoenix-core/src/it/java/org/apache/phoenix/end2end/SysTableNamespaceMappedStatsCollectorIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SysTableNamespaceMappedStatsCollectorIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SysTableNamespaceMappedStatsCollectorIT.java
index ea5f32f..4830189 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SysTableNamespaceMappedStatsCollectorIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SysTableNamespaceMappedStatsCollectorIT.java
@@ -22,6 +22,7 @@ import java.util.Collection;
 import java.util.Map;
 
 import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.schema.stats.StatsCollectorIT;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.junit.BeforeClass;
 import org.junit.runners.Parameterized.Parameters;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/59e49f78/phoenix-core/src/it/java/org/apache/phoenix/schema/stats/StatsCollectorIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/schema/stats/StatsCollectorIT.java b/phoenix-core/src/it/java/org/apache/phoenix/schema/stats/StatsCollectorIT.java
new file mode 100644
index 0000000..76e3e8e
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/schema/stats/StatsCollectorIT.java
@@ -0,0 +1,831 @@
+/*
+ * 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.schema.stats;
+
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_STATS_TABLE;
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.apache.phoenix.util.TestUtil.getAllSplits;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.sql.Array;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Random;
+
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.coprocessor.UngroupedAggregateRegionObserver;
+import org.apache.phoenix.end2end.BaseUniqueNamesOwnClusterIT;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.query.BaseTest;
+import org.apache.phoenix.query.ConnectionQueryServices;
+import org.apache.phoenix.query.KeyRange;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableImpl;
+import org.apache.phoenix.schema.PTableKey;
+import org.apache.phoenix.util.MetaDataUtil;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.QueryUtil;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.TestUtil;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import com.google.common.collect.Maps;
+
+@RunWith(Parameterized.class)
+public abstract class StatsCollectorIT extends BaseUniqueNamesOwnClusterIT {
+    private final String tableDDLOptions;
+    private final boolean columnEncoded;
+    private String tableName;
+    private String schemaName;
+    private String fullTableName;
+    private String physicalTableName;
+    private final boolean userTableNamespaceMapped;
+    private final boolean mutable;
+    private static final int defaultGuidePostWidth = 20;
+    
+    protected StatsCollectorIT(boolean mutable, boolean transactional, boolean userTableNamespaceMapped, boolean columnEncoded) {
+        StringBuilder sb = new StringBuilder();
+        if (transactional) {
+            sb.append("TRANSACTIONAL=true");
+        }
+        if (!columnEncoded) {
+            if (sb.length()>0) {
+                sb.append(",");
+            }
+            sb.append("COLUMN_ENCODED_BYTES=0");
+        } else {
+            if (sb.length()>0) {
+                sb.append(",");
+            }
+            sb.append("COLUMN_ENCODED_BYTES=4");
+        }
+        if (!mutable) {
+            if (sb.length()>0) {
+                sb.append(",");
+            }
+            sb.append("IMMUTABLE_ROWS=true");
+            if (!columnEncoded) {
+                sb.append(",IMMUTABLE_STORAGE_SCHEME="+PTableImpl.ImmutableStorageScheme.ONE_CELL_PER_COLUMN);
+            }
+        }
+        this.tableDDLOptions = sb.toString();
+        this.userTableNamespaceMapped = userTableNamespaceMapped;
+        this.columnEncoded = columnEncoded;
+        this.mutable = mutable;
+    }
+
+    @BeforeClass
+    public static void doSetup() throws Exception {
+        // enable name space mapping at global level on both client and server side
+        Map<String, String> serverProps = Maps.newHashMapWithExpectedSize(7);
+        serverProps.put(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, "true");
+        serverProps.put(QueryServices.STATS_GUIDEPOST_WIDTH_BYTES_ATTRIB, Long.toString(defaultGuidePostWidth));
+        Map<String, String> clientProps = Maps.newHashMapWithExpectedSize(2);
+        clientProps.put(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, "true");
+        clientProps.put(QueryServices.STATS_GUIDEPOST_WIDTH_BYTES_ATTRIB, Long.toString(defaultGuidePostWidth));
+        setUpTestDriver(new ReadOnlyProps(serverProps.entrySet().iterator()), new ReadOnlyProps(clientProps.entrySet().iterator()));
+    }
+    
+    @Before
+    public void generateTableNames() throws SQLException {
+        schemaName = generateUniqueName();
+        if (userTableNamespaceMapped) {
+            try (Connection conn = getConnection()) {
+                conn.createStatement().execute("CREATE SCHEMA " + schemaName);
+            }
+        }
+        tableName = "T_" + generateUniqueName();
+        fullTableName = SchemaUtil.getTableName(schemaName, tableName);
+        physicalTableName = SchemaUtil.getPhysicalHBaseTableName(schemaName, tableName, userTableNamespaceMapped).getString();
+    }
+
+    private Connection getConnection() throws SQLException {
+        return getConnection(Integer.MAX_VALUE);
+    }
+
+    private Connection getConnection(Integer statsUpdateFreq) throws SQLException {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        props.setProperty(QueryServices.EXPLAIN_CHUNK_COUNT_ATTRIB, Boolean.TRUE.toString());
+        props.setProperty(QueryServices.EXPLAIN_ROW_COUNT_ATTRIB, Boolean.TRUE.toString());
+        props.setProperty(QueryServices.STATS_UPDATE_FREQ_MS_ATTRIB, Integer.toString(statsUpdateFreq));
+        // enable/disable namespace mapping at connection level
+        props.setProperty(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.toString(userTableNamespaceMapped));
+        return DriverManager.getConnection(getUrl(), props);
+    }
+    
+    @Test
+    public void testUpdateEmptyStats() throws Exception {
+        Connection conn = getConnection();
+        conn.setAutoCommit(true);
+        conn.createStatement().execute(
+                "CREATE TABLE " + fullTableName +" ( k CHAR(1) PRIMARY KEY )"  + tableDDLOptions);
+        conn.createStatement().execute("UPDATE STATISTICS " + fullTableName);
+        ResultSet rs = conn.createStatement().executeQuery("EXPLAIN SELECT * FROM " + fullTableName);
+        String explainPlan = QueryUtil.getExplainPlan(rs);
+        assertEquals(
+                "CLIENT 1-CHUNK 0 ROWS 20 BYTES PARALLEL 1-WAY FULL SCAN OVER " + physicalTableName + "\n" +
+                "    SERVER FILTER BY FIRST KEY ONLY",
+                explainPlan);
+        conn.close();
+    }
+    
+    @Test
+    public void testSomeUpdateEmptyStats() throws Exception {
+        Connection conn = getConnection();
+        conn.setAutoCommit(true);
+        conn.createStatement().execute(
+                "CREATE TABLE " + fullTableName +" ( k VARCHAR PRIMARY KEY, a.v1 VARCHAR, b.v2 VARCHAR ) " + tableDDLOptions + (tableDDLOptions.isEmpty() ? "" : ",") + "SALT_BUCKETS = 3");
+        conn.createStatement().execute("UPSERT INTO " + fullTableName + "(k,v1) VALUES('a','123456789')");
+        conn.createStatement().execute("UPDATE STATISTICS " + fullTableName);
+                
+        ResultSet rs;
+        String explainPlan;
+        rs = conn.createStatement().executeQuery("EXPLAIN SELECT v2 FROM " + fullTableName + " WHERE v2='foo'");
+        explainPlan = QueryUtil.getExplainPlan(rs);
+        // if we are using the ONE_CELL_PER_COLUMN_FAMILY storage scheme, we will have the single kv even though there are no values for col family v2 
+        String stats = columnEncoded && !mutable  ? "4-CHUNK 1 ROWS 38 BYTES" : "3-CHUNK 0 ROWS 20 BYTES";
+        assertEquals(
+                "CLIENT " + stats + " PARALLEL 3-WAY FULL SCAN OVER " + physicalTableName + "\n" +
+                "    SERVER FILTER BY B.V2 = 'foo'\n" + 
+                "CLIENT MERGE SORT",
+                explainPlan);
+        rs = conn.createStatement().executeQuery("EXPLAIN SELECT * FROM " + fullTableName);
+        explainPlan = QueryUtil.getExplainPlan(rs);
+        assertEquals(
+                "CLIENT 4-CHUNK 1 ROWS " + (columnEncoded ? "28" : "34") + " BYTES PARALLEL 3-WAY FULL SCAN OVER " + physicalTableName + "\n" +
+                "CLIENT MERGE SORT",
+                explainPlan);
+        rs = conn.createStatement().executeQuery("EXPLAIN SELECT * FROM " + fullTableName + " WHERE k = 'a'");
+        explainPlan = QueryUtil.getExplainPlan(rs);
+        assertEquals(
+                "CLIENT 1-CHUNK 1 ROWS " + (columnEncoded ? "204" : "202") + " BYTES PARALLEL 1-WAY POINT LOOKUP ON 1 KEY OVER " + physicalTableName + "\n" +
+                "CLIENT MERGE SORT",
+                explainPlan);
+        
+        conn.close();
+    }
+    
+    @Test
+    public void testUpdateStats() throws SQLException, IOException,
+			InterruptedException {
+		Connection conn;
+        PreparedStatement stmt;
+        ResultSet rs;
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        conn = getConnection();
+        conn.createStatement().execute(
+                "CREATE TABLE " + fullTableName +" ( k VARCHAR, a_string_array VARCHAR(100) ARRAY[4], b_string_array VARCHAR(100) ARRAY[4] \n"
+                        + " CONSTRAINT pk PRIMARY KEY (k, b_string_array DESC))"
+                		+ tableDDLOptions );
+        String[] s;
+        Array array;
+        conn = upsertValues(props, fullTableName);
+        // CAll the update statistics query here. If already major compaction has run this will not get executed.
+        stmt = conn.prepareStatement("UPDATE STATISTICS " + fullTableName);
+        stmt.execute();
+        stmt = upsertStmt(conn, fullTableName);
+        stmt.setString(1, "z");
+        s = new String[] { "xyz", "def", "ghi", "jkll", null, null, "xxx" };
+        array = conn.createArrayOf("VARCHAR", s);
+        stmt.setArray(2, array);
+        s = new String[] { "zya", "def", "ghi", "jkll", null, null, null, "xxx" };
+        array = conn.createArrayOf("VARCHAR", s);
+        stmt.setArray(3, array);
+        stmt.execute();
+        stmt = conn.prepareStatement("UPDATE STATISTICS " + fullTableName);
+        stmt.execute();
+        rs = conn.createStatement().executeQuery("SELECT k FROM " + fullTableName);
+        assertTrue(rs.next());
+        conn.close();
+    }
+
+    private void testNoDuplicatesAfterUpdateStats(String splitKey) throws Throwable {
+        Connection conn = getConnection();
+        PreparedStatement stmt;
+        ResultSet rs;
+        conn.createStatement()
+                .execute("CREATE TABLE " + fullTableName
+                        + " ( k VARCHAR, c1.a bigint,c2.b bigint CONSTRAINT pk PRIMARY KEY (k))"+ tableDDLOptions
+                        + (splitKey != null ? " split on (" + splitKey + ")" : "") );
+        conn.createStatement().execute("upsert into " + fullTableName + " values ('abc',1,3)");
+        conn.createStatement().execute("upsert into " + fullTableName + " values ('def',2,4)");
+        conn.commit();
+        conn.createStatement().execute("UPDATE STATISTICS " + fullTableName);
+        rs = conn.createStatement().executeQuery("SELECT k FROM " + fullTableName + " order by k desc");
+        assertTrue(rs.next());
+        assertEquals("def", rs.getString(1));
+        assertTrue(rs.next());
+        assertEquals("abc", rs.getString(1));
+        assertTrue(!rs.next());
+        conn.close();
+    }
+
+    @Test
+    public void testNoDuplicatesAfterUpdateStatsWithSplits() throws Throwable {
+        testNoDuplicatesAfterUpdateStats("'abc','def'");
+    }
+
+    @Test
+    public void testNoDuplicatesAfterUpdateStatsWithDesc() throws Throwable {
+        testNoDuplicatesAfterUpdateStats(null);
+    }
+
+    @Test
+    public void testUpdateStatsWithMultipleTables() throws Throwable {
+        String fullTableName2 = SchemaUtil.getTableName(schemaName, "T_" + generateUniqueName());
+        Connection conn;
+        PreparedStatement stmt;
+        ResultSet rs;
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        conn = getConnection();
+        conn.createStatement().execute(
+                "CREATE TABLE " + fullTableName +" ( k VARCHAR, a_string_array VARCHAR(100) ARRAY[4], b_string_array VARCHAR(100) ARRAY[4] \n"
+                        + " CONSTRAINT pk PRIMARY KEY (k, b_string_array DESC))" + tableDDLOptions );
+        conn.createStatement().execute(
+                "CREATE TABLE " + fullTableName2 +" ( k VARCHAR, a_string_array VARCHAR(100) ARRAY[4], b_string_array VARCHAR(100) ARRAY[4] \n"
+                        + " CONSTRAINT pk PRIMARY KEY (k, b_string_array DESC))" + tableDDLOptions );
+        String[] s;
+        Array array;
+        conn = upsertValues(props, fullTableName);
+        conn = upsertValues(props, fullTableName2);
+        // CAll the update statistics query here
+        stmt = conn.prepareStatement("UPDATE STATISTICS "+fullTableName);
+        stmt.execute();
+        stmt = conn.prepareStatement("UPDATE STATISTICS "+fullTableName2);
+        stmt.execute();
+        stmt = upsertStmt(conn, fullTableName);
+        stmt.setString(1, "z");
+        s = new String[] { "xyz", "def", "ghi", "jkll", null, null, "xxx" };
+        array = conn.createArrayOf("VARCHAR", s);
+        stmt.setArray(2, array);
+        s = new String[] { "zya", "def", "ghi", "jkll", null, null, null, "xxx" };
+        array = conn.createArrayOf("VARCHAR", s);
+        stmt.setArray(3, array);
+        stmt.execute();
+        stmt = upsertStmt(conn, fullTableName2);
+        stmt.setString(1, "z");
+        s = new String[] { "xyz", "def", "ghi", "jkll", null, null, "xxx" };
+        array = conn.createArrayOf("VARCHAR", s);
+        stmt.setArray(2, array);
+        s = new String[] { "zya", "def", "ghi", "jkll", null, null, null, "xxx" };
+        array = conn.createArrayOf("VARCHAR", s);
+        stmt.setArray(3, array);
+        stmt.execute();
+        conn.close();
+        conn = getConnection();
+        // This analyze would not work
+        stmt = conn.prepareStatement("UPDATE STATISTICS "+fullTableName2);
+        stmt.execute();
+        rs = conn.createStatement().executeQuery("SELECT k FROM "+fullTableName2);
+        assertTrue(rs.next());
+        conn.close();
+    }
+
+    private Connection upsertValues(Properties props, String tableName) throws SQLException, IOException,
+            InterruptedException {
+        Connection conn;
+        PreparedStatement stmt;
+        conn = getConnection();
+        stmt = upsertStmt(conn, tableName);
+        stmt.setString(1, "a");
+        String[] s = new String[] { "abc", "def", "ghi", "jkll", null, null, "xxx" };
+        Array array = conn.createArrayOf("VARCHAR", s);
+        stmt.setArray(2, array);
+        s = new String[] { "abc", "def", "ghi", "jkll", null, null, null, "xxx" };
+        array = conn.createArrayOf("VARCHAR", s);
+        stmt.setArray(3, array);
+        stmt.execute();
+        conn.commit();
+        stmt = upsertStmt(conn, tableName);
+        stmt.setString(1, "b");
+        s = new String[] { "xyz", "def", "ghi", "jkll", null, null, "xxx" };
+        array = conn.createArrayOf("VARCHAR", s);
+        stmt.setArray(2, array);
+        s = new String[] { "zya", "def", "ghi", "jkll", null, null, null, "xxx" };
+        array = conn.createArrayOf("VARCHAR", s);
+        stmt.setArray(3, array);
+        stmt.execute();
+        conn.commit();
+        stmt = upsertStmt(conn, tableName);
+        stmt.setString(1, "c");
+        s = new String[] { "xyz", "def", "ghi", "jkll", null, null, "xxx" };
+        array = conn.createArrayOf("VARCHAR", s);
+        stmt.setArray(2, array);
+        s = new String[] { "zya", "def", "ghi", "jkll", null, null, null, "xxx" };
+        array = conn.createArrayOf("VARCHAR", s);
+        stmt.setArray(3, array);
+        stmt.execute();
+        conn.commit();
+        stmt = upsertStmt(conn, tableName);
+        stmt.setString(1, "d");
+        s = new String[] { "xyz", "def", "ghi", "jkll", null, null, "xxx" };
+        array = conn.createArrayOf("VARCHAR", s);
+        stmt.setArray(2, array);
+        s = new String[] { "zya", "def", "ghi", "jkll", null, null, null, "xxx" };
+        array = conn.createArrayOf("VARCHAR", s);
+        stmt.setArray(3, array);
+        stmt.execute();
+        conn.commit();
+        stmt = upsertStmt(conn, tableName);
+        stmt.setString(1, "b");
+        s = new String[] { "xyz", "def", "ghi", "jkll", null, null, "xxx" };
+        array = conn.createArrayOf("VARCHAR", s);
+        stmt.setArray(2, array);
+        s = new String[] { "zya", "def", "ghi", "jkll", null, null, null, "xxx" };
+        array = conn.createArrayOf("VARCHAR", s);
+        stmt.setArray(3, array);
+        stmt.execute();
+        conn.commit();
+        stmt = upsertStmt(conn, tableName);
+        stmt.setString(1, "e");
+        s = new String[] { "xyz", "def", "ghi", "jkll", null, null, "xxx" };
+        array = conn.createArrayOf("VARCHAR", s);
+        stmt.setArray(2, array);
+        s = new String[] { "zya", "def", "ghi", "jkll", null, null, null, "xxx" };
+        array = conn.createArrayOf("VARCHAR", s);
+        stmt.setArray(3, array);
+        stmt.execute();
+        conn.commit();
+        return conn;
+    }
+
+    private PreparedStatement upsertStmt(Connection conn, String tableName) throws SQLException {
+        PreparedStatement stmt;
+        stmt = conn.prepareStatement("UPSERT INTO " + tableName + " VALUES(?,?,?)");
+        return stmt;
+    }
+
+    private void compactTable(Connection conn, String tableName) throws Exception {
+        TestUtil.doMajorCompaction(conn, tableName);
+    }
+    
+    @Test
+    @Ignore //TODO remove this once  https://issues.apache.org/jira/browse/TEPHRA-208 is fixed
+    public void testCompactUpdatesStats() throws Exception {
+        testCompactUpdatesStats(0, fullTableName);
+    }
+    
+    @Test
+    @Ignore //TODO remove this once  https://issues.apache.org/jira/browse/TEPHRA-208 is fixed
+    public void testCompactUpdatesStatsWithMinStatsUpdateFreq() throws Exception {
+        testCompactUpdatesStats(QueryServicesOptions.DEFAULT_STATS_UPDATE_FREQ_MS, fullTableName);
+    }
+    
+    private static void invalidateStats(Connection conn, String tableName) throws SQLException {
+        PTable ptable = conn.unwrap(PhoenixConnection.class)
+                .getMetaDataCache().getTableRef(new PTableKey(null, tableName))
+                .getTable();
+        byte[] name = ptable.getPhysicalName().getBytes();
+        conn.unwrap(PhoenixConnection.class).getQueryServices().invalidateStats(new GuidePostsKey(name, SchemaUtil.getEmptyColumnFamily(ptable)));
+    }
+    
+    private void testCompactUpdatesStats(Integer statsUpdateFreq, String tableName) throws Exception {
+        int nRows = 10;
+        Connection conn = getConnection(statsUpdateFreq);
+        PreparedStatement stmt;
+        conn.createStatement().execute("CREATE TABLE " + tableName + "(k CHAR(1) PRIMARY KEY, v INTEGER, w INTEGER) "
+                + (!tableDDLOptions.isEmpty() ? tableDDLOptions + "," : "") 
+                + HColumnDescriptor.KEEP_DELETED_CELLS + "=" + Boolean.FALSE);
+        stmt = conn.prepareStatement("UPSERT INTO " + tableName + " VALUES(?,?,?)");
+        for (int i = 0; i < nRows; i++) {
+            stmt.setString(1, Character.toString((char) ('a' + i)));
+            stmt.setInt(2, i);
+            stmt.setInt(3, i);
+            stmt.executeUpdate();
+        }
+        conn.commit();
+        
+        compactTable(conn, physicalTableName);
+        
+        if (statsUpdateFreq != 0) {
+            invalidateStats(conn, tableName);
+        } else {
+            // Confirm that when we have a non zero STATS_UPDATE_FREQ_MS_ATTRIB, after we run
+            // UPDATATE STATISTICS, the new statistics are faulted in as expected.
+            List<KeyRange>keyRanges = getAllSplits(conn, tableName);
+            assertNotEquals(nRows+1, keyRanges.size());
+            // If we've set MIN_STATS_UPDATE_FREQ_MS_ATTRIB, an UPDATE STATISTICS will invalidate the cache
+            // and forcing the new stats to be pulled over.
+            int rowCount = conn.createStatement().executeUpdate("UPDATE STATISTICS " + tableName);
+            assertEquals(10, rowCount);
+        }
+        List<KeyRange>keyRanges = getAllSplits(conn, tableName);
+        assertEquals(nRows+1, keyRanges.size());
+        
+        int nDeletedRows = conn.createStatement().executeUpdate("DELETE FROM " + tableName + " WHERE V < " + nRows / 2);
+        conn.commit();
+        assertEquals(5, nDeletedRows);
+        
+        Scan scan = new Scan();
+        scan.setRaw(true);
+        PhoenixConnection phxConn = conn.unwrap(PhoenixConnection.class);
+        try (Table htable = phxConn.getQueryServices().getTable(Bytes.toBytes(tableName))) {
+            ResultScanner scanner = htable.getScanner(scan);
+            Result result;
+            while ((result = scanner.next())!=null) {
+                System.out.println(result);
+            }
+        }
+        
+        compactTable(conn, physicalTableName);
+        
+        scan = new Scan();
+        scan.setRaw(true);
+        phxConn = conn.unwrap(PhoenixConnection.class);
+        try (Table htable = phxConn.getQueryServices().getTable(Bytes.toBytes(tableName))) {
+            ResultScanner scanner = htable.getScanner(scan);
+            Result result;
+            while ((result = scanner.next())!=null) {
+                System.out.println(result);
+            }
+        }
+        
+        if (statsUpdateFreq != 0) {
+            invalidateStats(conn, tableName);
+        } else {
+            assertEquals(nRows+1, keyRanges.size());
+            // If we've set STATS_UPDATE_FREQ_MS_ATTRIB, an UPDATE STATISTICS will invalidate the cache
+            // and force us to pull over the new stats
+            int rowCount = conn.createStatement().executeUpdate("UPDATE STATISTICS " + tableName);
+            assertEquals(5, rowCount);
+        }
+        keyRanges = getAllSplits(conn, tableName);
+        assertEquals(nRows/2+1, keyRanges.size());
+        ResultSet rs = conn.createStatement().executeQuery("SELECT SUM(GUIDE_POSTS_ROW_COUNT) FROM "
+                + "\""+ SYSTEM_CATALOG_SCHEMA + "\".\"" + SYSTEM_STATS_TABLE + "\"" + " WHERE PHYSICAL_NAME='" + physicalTableName + "'");
+        rs.next();
+        assertEquals(nRows - nDeletedRows, rs.getLong(1));
+    }
+
+    @Test
+    public void testWithMultiCF() throws Exception {
+        int nRows = 20;
+        Connection conn = getConnection(0);
+        PreparedStatement stmt;
+        conn.createStatement().execute(
+                "CREATE TABLE " + fullTableName
+                        + "(k VARCHAR PRIMARY KEY, a.v INTEGER, b.v INTEGER, c.v INTEGER NULL, d.v INTEGER NULL) "
+                        + tableDDLOptions );
+        stmt = conn.prepareStatement("UPSERT INTO " + fullTableName + " VALUES(?,?, ?, ?, ?)");
+        byte[] val = new byte[250];
+        for (int i = 0; i < nRows; i++) {
+            stmt.setString(1, Character.toString((char)('a' + i)) + Bytes.toString(val));
+            stmt.setInt(2, i);
+            stmt.setInt(3, i);
+            stmt.setInt(4, i);
+            stmt.setInt(5, i);
+            stmt.executeUpdate();
+        }
+        conn.commit();
+        stmt = conn.prepareStatement("UPSERT INTO " + fullTableName + "(k, c.v, d.v) VALUES(?,?,?)");
+        for (int i = 0; i < 5; i++) {
+            stmt.setString(1, Character.toString((char)('a' + 'z' + i)) + Bytes.toString(val));
+            stmt.setInt(2, i);
+            stmt.setInt(3, i);
+            stmt.executeUpdate();
+        }
+        conn.commit();
+
+        ResultSet rs;
+        TestUtil.analyzeTable(conn, fullTableName);
+        List<KeyRange> keyRanges = getAllSplits(conn, fullTableName);
+        assertEquals(26, keyRanges.size());
+        rs = conn.createStatement().executeQuery("EXPLAIN SELECT * FROM " + fullTableName);
+        assertEquals("CLIENT 26-CHUNK 25 ROWS " + (columnEncoded ? ( mutable ? "12530" : "13902" ) : "12420") + " BYTES PARALLEL 1-WAY FULL SCAN OVER " + physicalTableName,
+                QueryUtil.getExplainPlan(rs));
+
+        ConnectionQueryServices services = conn.unwrap(PhoenixConnection.class).getQueryServices();
+        List<HRegionLocation> regions = services.getAllTableRegions(Bytes.toBytes(physicalTableName));
+        assertEquals(1, regions.size());
+
+        TestUtil.analyzeTable(conn, fullTableName);
+        String query = "UPDATE STATISTICS " + fullTableName + " SET \""
+                + QueryServices.STATS_GUIDEPOST_WIDTH_BYTES_ATTRIB + "\"=" + Long.toString(1000);
+        conn.createStatement().execute(query);
+        keyRanges = getAllSplits(conn, fullTableName);
+        boolean oneCellPerColFamliyStorageScheme = !mutable && columnEncoded;
+        assertEquals(oneCellPerColFamliyStorageScheme ? 13 : 12, keyRanges.size());
+
+        rs = conn
+                .createStatement()
+                .executeQuery(
+                        "SELECT COLUMN_FAMILY,SUM(GUIDE_POSTS_ROW_COUNT),SUM(GUIDE_POSTS_WIDTH),COUNT(*) from \"SYSTEM\".STATS where PHYSICAL_NAME = '"
+                                + physicalTableName + "' GROUP BY COLUMN_FAMILY ORDER BY COLUMN_FAMILY");
+
+        assertTrue(rs.next());
+        assertEquals("A", rs.getString(1));
+        assertEquals(24, rs.getInt(2));
+        assertEquals(columnEncoded ? ( mutable ? 12252 : 13624 ) : 12144, rs.getInt(3));
+        assertEquals(oneCellPerColFamliyStorageScheme ? 12 : 11, rs.getInt(4));
+
+        assertTrue(rs.next());
+        assertEquals("B", rs.getString(1));
+        assertEquals(oneCellPerColFamliyStorageScheme ? 24 : 20, rs.getInt(2));
+        assertEquals(columnEncoded ? ( mutable ? 5600 : 6972 ) : 5540, rs.getInt(3));
+        assertEquals(oneCellPerColFamliyStorageScheme ? 6 : 5, rs.getInt(4));
+
+        assertTrue(rs.next());
+        assertEquals("C", rs.getString(1));
+        assertEquals(24, rs.getInt(2));
+        assertEquals(columnEncoded ? ( mutable ? 6724 : 6988 ) : 6652, rs.getInt(3));
+        assertEquals(6, rs.getInt(4));
+
+        assertTrue(rs.next());
+        assertEquals("D", rs.getString(1));
+        assertEquals(24, rs.getInt(2));
+        assertEquals(columnEncoded ? ( mutable ? 6724 : 6988 ) : 6652, rs.getInt(3));
+        assertEquals(6, rs.getInt(4));
+
+        assertFalse(rs.next());
+        
+        // Disable stats
+        conn.createStatement().execute("ALTER TABLE " + fullTableName + 
+                " SET " + PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH + "=0");
+        TestUtil.analyzeTable(conn, fullTableName);
+        // Assert that there are no more guideposts
+        rs = conn.createStatement().executeQuery("SELECT count(1) FROM " + PhoenixDatabaseMetaData.SYSTEM_STATS_NAME + 
+                " WHERE " + PhoenixDatabaseMetaData.PHYSICAL_NAME + "='" + physicalTableName + "' AND " + PhoenixDatabaseMetaData.COLUMN_FAMILY + " IS NOT NULL");
+        assertTrue(rs.next());
+        assertEquals(0, rs.getLong(1));
+        assertFalse(rs.next());
+        rs = conn.createStatement().executeQuery("EXPLAIN SELECT * FROM " + fullTableName);
+        assertEquals("CLIENT 1-CHUNK PARALLEL 1-WAY FULL SCAN OVER " + physicalTableName,
+                QueryUtil.getExplainPlan(rs));
+    }
+
+    @Test
+    public void testRowCountAndByteCounts() throws SQLException {
+        Connection conn = getConnection();
+        String ddl = "CREATE TABLE " + fullTableName + " (t_id VARCHAR NOT NULL,\n" + "k1 INTEGER NOT NULL,\n"
+                + "k2 INTEGER NOT NULL,\n" + "C3.k3 INTEGER,\n" + "C2.v1 VARCHAR,\n"
+                + "CONSTRAINT pk PRIMARY KEY (t_id, k1, k2)) " + tableDDLOptions + " split on ('e','j','o')";
+        conn.createStatement().execute(ddl);
+        String[] strings = { "a", "b", "c", "d", "e", "f", "g", "h", "i", "j", "k", "l", "m", "n", "o", "p", "q", "r",
+                "s", "t", "u", "v", "w", "x", "y", "z" };
+        for (int i = 0; i < 26; i++) {
+            conn.createStatement().execute(
+                    "UPSERT INTO " + fullTableName + " values('" + strings[i] + "'," + i + "," + (i + 1) + ","
+                            + (i + 2) + ",'" + strings[25 - i] + "')");
+        }
+        conn.commit();
+        ResultSet rs;
+        String query = "UPDATE STATISTICS " + fullTableName + " SET \""
+                + QueryServices.STATS_GUIDEPOST_WIDTH_BYTES_ATTRIB + "\"=" + Long.toString(20);
+        conn.createStatement().execute(query);
+        Random r = new Random();
+        int count = 0;
+        while (count < 4) {
+            int startIndex = r.nextInt(strings.length);
+            int endIndex = r.nextInt(strings.length - startIndex) + startIndex;
+            long rows = endIndex - startIndex;
+            long c2Bytes = rows * (columnEncoded ? ( mutable ? 37 : 48 ) : 35);
+            String physicalTableName = SchemaUtil.getPhysicalTableName(Bytes.toBytes(fullTableName), userTableNamespaceMapped).toString();
+            rs = conn.createStatement().executeQuery(
+                    "SELECT COLUMN_FAMILY,SUM(GUIDE_POSTS_ROW_COUNT),SUM(GUIDE_POSTS_WIDTH) from \"SYSTEM\".STATS where PHYSICAL_NAME = '"
+                            + physicalTableName + "' AND GUIDE_POST_KEY>= cast('" + strings[startIndex]
+                            + "' as varbinary) AND  GUIDE_POST_KEY<cast('" + strings[endIndex]
+                            + "' as varbinary) and COLUMN_FAMILY='C2' group by COLUMN_FAMILY");
+            if (startIndex < endIndex) {
+                assertTrue(rs.next());
+                assertEquals("C2", rs.getString(1));
+                assertEquals(rows, rs.getLong(2));
+                assertEquals(c2Bytes, rs.getLong(3));
+                count++;
+            }
+        }
+    }
+
+    @Test
+    public void testRowCountWhenNumKVsExceedCompactionScannerThreshold() throws Exception {
+        String tableName = generateUniqueName();
+        StringBuilder sb = new StringBuilder(200);
+        sb.append("CREATE TABLE " + tableName + "(PK1 VARCHAR NOT NULL, ");
+        int numRows = 10;
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            int compactionScannerKVThreshold =
+                    conn.unwrap(PhoenixConnection.class).getQueryServices().getConfiguration()
+                            .getInt(HConstants.COMPACTION_KV_MAX,
+                                HConstants.COMPACTION_KV_MAX_DEFAULT);
+            int numKvColumns = compactionScannerKVThreshold * 2;
+            for (int i = 1; i <= numKvColumns; i++) {
+                sb.append("KV" + i + " VARCHAR");
+                if (i < numKvColumns) {
+                    sb.append(", ");
+                }
+            }
+            sb.append(" CONSTRAINT PK PRIMARY KEY (PK1))");
+            String ddl = sb.toString();
+            conn.createStatement().execute(ddl);
+            sb = new StringBuilder(200);
+            sb.append("UPSERT INTO " + tableName + " VALUES (");
+            for (int i = 1; i <= numKvColumns + 1; i++) {
+                sb.append("?");
+                if (i < numKvColumns + 1) {
+                    sb.append(", ");
+                }
+            }
+            sb.append(")");
+            String dml = sb.toString();
+            PreparedStatement stmt = conn.prepareStatement(dml);
+            String keyValue = "KVVVVVV";
+            for (int j = 1; j <= numRows; j++) {
+                for (int i = 1; i <= numKvColumns + 1; i++) {
+                    if (i == 1) {
+                        stmt.setString(1, "" + j);
+                    } else {
+                        stmt.setString(i, keyValue);
+                    }
+                }
+                stmt.executeUpdate();
+            }
+            conn.commit();
+            conn.createStatement().execute("UPDATE STATISTICS " + tableName);
+            String q = "SELECT SUM(GUIDE_POSTS_ROW_COUNT) FROM SYSTEM.STATS WHERE PHYSICAL_NAME = '" + tableName + "'";
+            ResultSet rs = conn.createStatement().executeQuery(q);
+            rs.next();
+            assertEquals("Number of expected rows in stats table after update stats didn't match!", numRows, rs.getInt(1));
+            conn.createStatement().executeUpdate("DELETE FROM SYSTEM.STATS WHERE PHYSICAL_NAME = '" + tableName + "'");
+            conn.commit();
+            TestUtil.doMajorCompaction(conn, tableName);
+            q = "SELECT SUM(GUIDE_POSTS_ROW_COUNT) FROM SYSTEM.STATS WHERE PHYSICAL_NAME = '" + tableName + "'";
+            rs = conn.createStatement().executeQuery(q);
+            rs.next();
+            assertEquals("Number of expected rows in stats table after major compaction didn't match", numRows, rs.getInt(1));
+        }
+    }
+    
+    @Test
+    public void testEmptyGuidePostGeneratedWhenDataSizeLessThanGPWidth() throws Exception {
+        String tableName = generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            long guidePostWidth = 20000000;
+            conn.createStatement()
+                    .execute("CREATE TABLE " + tableName
+                            + " ( k INTEGER, c1.a bigint,c2.b bigint CONSTRAINT pk PRIMARY KEY (k)) GUIDE_POSTS_WIDTH="
+                            + guidePostWidth + ", SALT_BUCKETS = 4");
+            conn.createStatement().execute("upsert into " + tableName + " values (100,1,3)");
+            conn.createStatement().execute("upsert into " + tableName + " values (101,2,4)");
+            conn.commit();
+            conn.createStatement().execute("UPDATE STATISTICS " + tableName);
+            ConnectionQueryServices queryServices =
+                    conn.unwrap(PhoenixConnection.class).getQueryServices();
+            try (Table statsHTable =
+                    queryServices.getTable(
+                        SchemaUtil.getPhysicalName(PhoenixDatabaseMetaData.SYSTEM_STATS_NAME_BYTES,
+                            queryServices.getProps()).getName())) {
+                GuidePostsInfo gps =
+                        StatisticsUtil.readStatistics(statsHTable,
+                            new GuidePostsKey(Bytes.toBytes(tableName), Bytes.toBytes("C1")),
+                            HConstants.LATEST_TIMESTAMP);
+                assertTrue(gps.isEmptyGuidePost());
+                assertEquals(guidePostWidth, gps.getByteCounts()[0]);
+                assertTrue(gps.getGuidePostTimestamps()[0] > 0);
+                gps =
+                        StatisticsUtil.readStatistics(statsHTable,
+                            new GuidePostsKey(Bytes.toBytes(tableName), Bytes.toBytes("C2")),
+                            HConstants.LATEST_TIMESTAMP);
+                assertTrue(gps.isEmptyGuidePost());
+                assertEquals(guidePostWidth, gps.getByteCounts()[0]);
+                assertTrue(gps.getGuidePostTimestamps()[0] > 0);
+            }
+        }
+    }
+
+    @Test
+    public void testGuidePostWidthUsedInDefaultStatsCollector() throws Exception {
+        String baseTable = generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String ddl =
+                    "CREATE TABLE " + baseTable
+                            + " (k INTEGER PRIMARY KEY, a bigint, b bigint, c bigint) "
+                            + tableDDLOptions;
+            BaseTest.createTestTable(getUrl(), ddl, null, null);
+            conn.createStatement().execute("upsert into " + baseTable + " values (100,1,1,1)");
+            conn.createStatement().execute("upsert into " + baseTable + " values (101,2,2,2)");
+            conn.createStatement().execute("upsert into " + baseTable + " values (102,3,3,3)");
+            conn.createStatement().execute("upsert into " + baseTable + " values (103,4,4,4)");
+            conn.createStatement().execute("upsert into " + baseTable + " values (104,5,5,5)");
+            conn.createStatement().execute("upsert into " + baseTable + " values (105,6,6,6)");
+            conn.createStatement().execute("upsert into " + baseTable + " values (106,7,7,7)");
+            conn.createStatement().execute("upsert into " + baseTable + " values (107,8,8,8)");
+            conn.createStatement().execute("upsert into " + baseTable + " values (108,9,9,9)");
+            conn.createStatement().execute("upsert into " + baseTable + " values (109,10,10,10)");
+            conn.commit();
+            DefaultStatisticsCollector statsCollector = getDefaultStatsCollectorForTable(baseTable);
+            statsCollector.init();
+            assertEquals(defaultGuidePostWidth, statsCollector.getGuidePostDepth());
+
+            // ok let's create a global index now and see what guide post width is used for it
+            String globalIndex = "GI_" + generateUniqueName();
+            ddl = "CREATE INDEX " + globalIndex + " ON " + baseTable + " (a) INCLUDE (b) ";
+            conn.createStatement().execute(ddl);
+            statsCollector = getDefaultStatsCollectorForTable(globalIndex);
+            statsCollector.init();
+            assertEquals(defaultGuidePostWidth, statsCollector.getGuidePostDepth());
+
+            // let's check out local index too
+            String localIndex = "LI_" + generateUniqueName();
+            ddl = "CREATE LOCAL INDEX " + localIndex + " ON " + baseTable + " (b) INCLUDE (c) ";
+            conn.createStatement().execute(ddl);
+            // local indexes reside on the same table as base data table
+            statsCollector = getDefaultStatsCollectorForTable(baseTable);
+            statsCollector.init();
+            assertEquals(defaultGuidePostWidth, statsCollector.getGuidePostDepth());
+
+            // now let's create a view and an index on it and see what guide post width is used for
+            // it
+            String view = "V_" + generateUniqueName();
+            ddl = "CREATE VIEW " + view + " AS SELECT * FROM " + baseTable;
+            conn.createStatement().execute(ddl);
+            String viewIndex = "VI_" + generateUniqueName();
+            ddl = "CREATE INDEX " + viewIndex + " ON " + view + " (b)";
+            conn.createStatement().execute(ddl);
+            String viewIndexTableName = MetaDataUtil.getViewIndexTableName(baseTable);
+            statsCollector = getDefaultStatsCollectorForTable(viewIndexTableName);
+            statsCollector.init();
+            assertEquals(defaultGuidePostWidth, statsCollector.getGuidePostDepth());
+            /*
+             * Fantastic! Now let's change the guide post width of the base table. This should
+             * change the guide post width we are using in DefaultStatisticsCollector for all
+             * indexes too.
+             */
+            long newGpWidth = 500;
+            conn.createStatement()
+                    .execute("ALTER TABLE " + baseTable + " SET GUIDE_POSTS_WIDTH=" + newGpWidth);
+
+            // base table
+            statsCollector = getDefaultStatsCollectorForTable(baseTable);
+            statsCollector.init();
+            assertEquals(newGpWidth, statsCollector.getGuidePostDepth());
+
+            // global index table
+            statsCollector = getDefaultStatsCollectorForTable(globalIndex);
+            statsCollector.init();
+            assertEquals(newGpWidth, statsCollector.getGuidePostDepth());
+
+            // view index table
+            statsCollector = getDefaultStatsCollectorForTable(viewIndexTableName);
+            statsCollector.init();
+            assertEquals(newGpWidth, statsCollector.getGuidePostDepth());
+        }
+    }
+
+    private DefaultStatisticsCollector getDefaultStatsCollectorForTable(String tableName)
+            throws Exception {
+        RegionCoprocessorEnvironment env = getRegionEnvrionment(tableName);
+        return (DefaultStatisticsCollector) StatisticsCollectorFactory
+                .createStatisticsCollector(env, tableName, System.currentTimeMillis(), null, null);
+    }
+
+    private RegionCoprocessorEnvironment getRegionEnvrionment(String tableName)
+            throws IOException, InterruptedException {
+        return getUtility()
+                .getRSForFirstRegionInTable(TableName.valueOf(tableName))
+                .getOnlineRegionsLocalContext().iterator().next().getCoprocessorHost()
+                .findCoprocessorEnvironment(UngroupedAggregateRegionObserver.class.getName());
+    }
+}


[19/20] phoenix git commit: PHOENIX-4291 Addendum - Merge release script for mac and linux

Posted by ja...@apache.org.
PHOENIX-4291 Addendum - Merge release script for mac and linux


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

Branch: refs/heads/5.x-HBase-2.0
Commit: ee91a603fe1536e4d3fe8a243e28c6ffa8ab7537
Parents: 28dc119
Author: Mujtaba <mu...@apache.org>
Authored: Fri Nov 3 13:41:45 2017 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Thu Nov 9 12:54:23 2017 -0800

----------------------------------------------------------------------
 dev/make_rc.sh | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/ee91a603/dev/make_rc.sh
----------------------------------------------------------------------
diff --git a/dev/make_rc.sh b/dev/make_rc.sh
index 31cb9f9..638968c 100755
--- a/dev/make_rc.sh
+++ b/dev/make_rc.sh
@@ -81,7 +81,7 @@ cp bin/* $DIR_BIN;
 cp -R $DIR_PHERF_CONF $DIR_BIN;
 
 # Copy release docs
-cp README $DIR_REL_BIN_PATH;
+cp README* $DIR_REL_BIN_PATH;
 cp $DIR_DOCS/* $DIR_REL_BIN_PATH;
 
 # Copy examples


[18/20] phoenix git commit: PHOENIX-4291 Merge release script for mac and linux

Posted by ja...@apache.org.
PHOENIX-4291 Merge release script for mac and linux


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

Branch: refs/heads/5.x-HBase-2.0
Commit: 28dc119fedc9a2b5c2429da72693811727ef621f
Parents: 3d4023a
Author: Mujtaba <mu...@apache.org>
Authored: Fri Nov 3 11:55:25 2017 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Thu Nov 9 12:54:03 2017 -0800

----------------------------------------------------------------------
 dev/make_rc.sh        |  26 +++++++---
 dev/make_rc_on_mac.sh | 121 ---------------------------------------------
 2 files changed, 18 insertions(+), 129 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/28dc119f/dev/make_rc.sh
----------------------------------------------------------------------
diff --git a/dev/make_rc.sh b/dev/make_rc.sh
index 29227b0..31cb9f9 100755
--- a/dev/make_rc.sh
+++ b/dev/make_rc.sh
@@ -43,7 +43,7 @@ DIR_DOCS=dev/release_files
 
 # Verify no target exists
 mvn clean; rm -rf $DIR_REL_BASE;
-RESULT=$(find -iname target)
+RESULT=$(find . -iname target)
 
 if [ -z "$RESULT" ]
 then
@@ -73,7 +73,7 @@ mvn clean apache-rat:check package -DskipTests -Dcheckstyle.skip=true -q;
 rm -rf $(find . -type d -name archive-tmp);
 
 # Copy all phoenix-*.jars to release dir
-phx_jars=$(find -iwholename "./*/target/phoenix-*.jar")
+phx_jars=$(find . -iwholename "./*/target/phoenix-*.jar")
 cp $phx_jars $DIR_REL_BIN_PATH;
 
 # Copy bin
@@ -81,7 +81,7 @@ cp bin/* $DIR_BIN;
 cp -R $DIR_PHERF_CONF $DIR_BIN;
 
 # Copy release docs
-
+cp README $DIR_REL_BIN_PATH;
 cp $DIR_DOCS/* $DIR_REL_BIN_PATH;
 
 # Copy examples
@@ -97,10 +97,20 @@ echo "Now signing source and binary tars"
 # Sign
 function_sign() {
   phoenix_tar=$(find apache-phoenix-*.gz);
-  gpg --armor --output $phoenix_tar.asc --detach-sig $phoenix_tar;
-  md5sum -b $phoenix_tar > $phoenix_tar.md5;
-  sha512sum -b $phoenix_tar > $phoenix_tar.sha;
-  sha256sum -b $phoenix_tar >> $phoenix_tar.sha;
+
+  # if on MAC OS
+  if [[ "$OSTYPE" == "darwin"* ]]; then
+    gpg2 --armor --output $phoenix_tar.asc --detach-sig $phoenix_tar;
+    openssl md5 $phoenix_tar > $phoenix_tar.md5;
+    openssl dgst -sha512 $phoenix_tar > $phoenix_tar.sha;
+    openssl dgst -sha256 $phoenix_tar >> $phoenix_tar.sha;
+  # all other OS
+  else
+    gpg --armor --output $phoenix_tar.asc --detach-sig $phoenix_tar;
+    md5sum -b $phoenix_tar > $phoenix_tar.md5;
+    sha512sum -b $phoenix_tar > $phoenix_tar.sha;
+    sha256sum -b $phoenix_tar >> $phoenix_tar.sha;
+  fi
 }
 
 cd $DIR_REL_BIN_TAR_PATH; function_sign;
@@ -111,7 +121,7 @@ read -p "Do you want add tag for this RC in GIT? (Y for yes or any other key to
 if [[ $prompt =~ [yY](es)* ]]
 then
   echo "Tagging..."
-  read -p "Enter tag (Example 5.0.0-rc0):" prompt
+  read -p "Enter tag (Example 4.13.0-HBase-0.98-rc0):" prompt
   echo "Setting tag: $prompt";sleep 5s
   git tag -a $prompt -m "$prompt"; git push origin $prompt
   mv $DIR_REL_ROOT $DIR_REL_BASE/phoenix-$prompt

http://git-wip-us.apache.org/repos/asf/phoenix/blob/28dc119f/dev/make_rc_on_mac.sh
----------------------------------------------------------------------
diff --git a/dev/make_rc_on_mac.sh b/dev/make_rc_on_mac.sh
deleted file mode 100755
index 0b924f1..0000000
--- a/dev/make_rc_on_mac.sh
+++ /dev/null
@@ -1,121 +0,0 @@
-#!/bin/bash
-############################################################################
-#
-# 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.
-#
-############################################################################
-set -e
-
-echo "Script that assembles all you need to make an RC."
-echo "It generates source and binary tar in release directory"
-echo "Presumes that you can sign a release as described at https://www.apache.org/dev/release-signing.html"
-echo "Starting...";sleep 2s
-
-# Set directory variables
-DIR_ROOT="$(cd $(dirname $0);pwd)/.."
-cd $DIR_ROOT
-PHOENIX="$(xmllint --xpath "//*[local-name()='project']/*[local-name()='version']/text()" pom.xml)"
-DIR_REL_BASE=$DIR_ROOT/release
-DIR_REL_ROOT=$DIR_REL_BASE/apache-phoenix-$PHOENIX
-DIR_REL_BIN=apache-phoenix-$PHOENIX-bin
-DIR_REL_BIN_PATH=$DIR_REL_ROOT/$DIR_REL_BIN
-REL_SRC=apache-phoenix-$PHOENIX-src
-DIR_REL_SRC_TAR_PATH=$DIR_REL_ROOT/src
-DIR_REL_BIN_TAR_PATH=$DIR_REL_ROOT/bin
-DIR_BIN=$DIR_REL_BIN_PATH/bin
-DIR_PHERF_CONF=phoenix-pherf/config
-DIR_EXAMPLES=$DIR_REL_BIN_PATH/examples
-DIR_DOCS=dev/release_files
-
-# Verify no target exists
-mvn clean; rm -rf $DIR_REL_BASE;
-RESULT=$(find . -iname target)
-
-if [ -z "$RESULT" ]
-then
-  echo "Verified target directory does not exist.";
-else
-  echo "Target directory exists at: $RESULT. Please use a clean repo.";
-  exit -1;
-fi
-
-# Generate src tar
-ln -s . $REL_SRC; tar cvzf $REL_SRC.tar.gz --exclude="$REL_SRC/$REL_SRC" $REL_SRC/*; rm $REL_SRC;
-
-# Generate directory structure
-mkdir $DIR_REL_BASE;
-mkdir $DIR_REL_ROOT;
-mkdir $DIR_REL_BIN_PATH;
-mkdir $DIR_REL_BIN_TAR_PATH;
-mkdir $DIR_REL_SRC_TAR_PATH;
-mkdir $DIR_EXAMPLES;
-mkdir $DIR_BIN;
-
-# Move src tar
-mv $REL_SRC.tar.gz $DIR_REL_SRC_TAR_PATH;
-
-# Copy common jars
-mvn clean apache-rat:check package -DskipTests -Dcheckstyle.skip=true -q;
-rm -rf $(find . -type d -name archive-tmp);
-
-# Copy all phoenix-*.jars to release dir
-phx_jars=$(find . -iwholename "./*/target/phoenix-*.jar")
-cp $phx_jars $DIR_REL_BIN_PATH;
-
-# Copy bin
-cp bin/* $DIR_BIN;
-cp -R $DIR_PHERF_CONF $DIR_BIN;
-
-# Copy release docs
-
-cp $DIR_DOCS/* $DIR_REL_BIN_PATH;
-
-# Copy examples
-cp -r examples/* $DIR_EXAMPLES
-
-# Generate bin tar
-tar cvzf $DIR_REL_BIN_TAR_PATH/$DIR_REL_BIN.tar.gz -C $DIR_REL_ROOT apache-phoenix-$PHOENIX-bin;
-rm -rf $DIR_REL_BIN_PATH;
-
-echo "DONE generating binary and source tars in release directory."
-echo "Now signing source and binary tars"
-
-# Sign
-function_sign() {
-  phoenix_tar=$(find apache-phoenix-*.gz);
-  gpg2 --armor --output $phoenix_tar.asc --detach-sig $phoenix_tar;
-  openssl md5 $phoenix_tar > $phoenix_tar.md5;
-  openssl dgst -sha512 $phoenix_tar > $phoenix_tar.sha;
-  openssl dgst -sha256 $phoenix_tar >> $phoenix_tar.sha;
-}
-
-cd $DIR_REL_BIN_TAR_PATH; function_sign;
-cd $DIR_REL_SRC_TAR_PATH; function_sign;
-
-# Tag
-read -p "Do you want add tag for this RC in GIT? (Y for yes or any other key to continue)" prompt
-if [[ $prompt =~ [yY](es)* ]]
-then
-  echo "Tagging..."
-  read -p "Enter tag (Example 5.0.0-rc0):" prompt
-  echo "Setting tag: $prompt";sleep 5s
-  git tag -a $prompt -m "$prompt"; git push origin $prompt
-  mv $DIR_REL_ROOT $DIR_REL_BASE/phoenix-$prompt
-fi
-
-echo "DONE."
-echo "If all looks good in release directory then commit RC at https://dist.apache.org/repos/dist/dev/phoenix"


[17/20] phoenix git commit: PHOENIX-4349 Update version to 4.13.0

Posted by ja...@apache.org.
PHOENIX-4349 Update version to 4.13.0


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

Branch: refs/heads/5.x-HBase-2.0
Commit: 3d4023a5805e7a556965ee053bca37b365ef47de
Parents: 4953876
Author: James Taylor <jt...@salesforce.com>
Authored: Fri Nov 3 09:26:58 2017 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Thu Nov 9 12:53:24 2017 -0800

----------------------------------------------------------------------
 .../java/org/apache/phoenix/coprocessor/MetaDataProtocol.java  | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/3d4023a5/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
index c4ecc3f..fe11ec7 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
@@ -63,7 +63,7 @@ import com.google.protobuf.ByteString;
  */
 public abstract class MetaDataProtocol extends MetaDataService {
     public static final int PHOENIX_MAJOR_VERSION = 4;
-    public static final int PHOENIX_MINOR_VERSION = 12;
+    public static final int PHOENIX_MINOR_VERSION = 13;
     public static final int PHOENIX_PATCH_NUMBER = 0;
     public static final int PHOENIX_VERSION =
             VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER);
@@ -92,8 +92,9 @@ public abstract class MetaDataProtocol extends MetaDataService {
     public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_11_0 = MIN_TABLE_TIMESTAMP + 27;
     // Since there's no upgrade code, keep the version the same as the previous version
     public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_12_0 = MIN_SYSTEM_TABLE_TIMESTAMP_4_11_0;
+    public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_13_0 = MIN_SYSTEM_TABLE_TIMESTAMP_4_11_0;
     // MIN_SYSTEM_TABLE_TIMESTAMP needs to be set to the max of all the MIN_SYSTEM_TABLE_TIMESTAMP_* constants
-    public static final long MIN_SYSTEM_TABLE_TIMESTAMP = MIN_SYSTEM_TABLE_TIMESTAMP_4_12_0;
+    public static final long MIN_SYSTEM_TABLE_TIMESTAMP = MIN_SYSTEM_TABLE_TIMESTAMP_4_13_0;
     
     // ALWAYS update this map whenever rolling out a new release (major, minor or patch release). 
     // Key is the SYSTEM.CATALOG timestamp for the version and value is the version string.
@@ -112,6 +113,7 @@ public abstract class MetaDataProtocol extends MetaDataService {
         TIMESTAMP_VERSION_MAP.put(MIN_SYSTEM_TABLE_TIMESTAMP_4_10_0, "4.10.x");
         TIMESTAMP_VERSION_MAP.put(MIN_SYSTEM_TABLE_TIMESTAMP_4_11_0, "4.11.x");
         TIMESTAMP_VERSION_MAP.put(MIN_SYSTEM_TABLE_TIMESTAMP_4_12_0, "4.12.x");
+        TIMESTAMP_VERSION_MAP.put(MIN_SYSTEM_TABLE_TIMESTAMP_4_13_0, "4.13.x");
     }
     
     public static final String CURRENT_CLIENT_VERSION = PHOENIX_MAJOR_VERSION + "." + PHOENIX_MINOR_VERSION + "." + PHOENIX_PATCH_NUMBER; 


[20/20] phoenix git commit: PHOENIX-4351 Add i18n-util to bin LICENSE file and to dependencyManagement

Posted by ja...@apache.org.
PHOENIX-4351 Add i18n-util to bin LICENSE file and to dependencyManagement


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

Branch: refs/heads/5.x-HBase-2.0
Commit: 0454e42112fa90b5c23c0d47be98e7ed9ad4f6c3
Parents: ee91a60
Author: Josh Elser <el...@apache.org>
Authored: Mon Nov 6 15:21:35 2017 -0500
Committer: James Taylor <jt...@salesforce.com>
Committed: Thu Nov 9 12:54:44 2017 -0800

----------------------------------------------------------------------
 dev/release_files/LICENSE | 2 ++
 phoenix-core/pom.xml      | 3 +--
 pom.xml                   | 5 +++++
 3 files changed, 8 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/0454e421/dev/release_files/LICENSE
----------------------------------------------------------------------
diff --git a/dev/release_files/LICENSE b/dev/release_files/LICENSE
index a72ce86..0fd0255 100644
--- a/dev/release_files/LICENSE
+++ b/dev/release_files/LICENSE
@@ -254,6 +254,8 @@ Janino Compiler (https://github.com/janino-compiler/janino)
 
 Hamcrest-core 1.3 (http://www.hamcrest.org) Copyright (c) 2000-2006, www.hamcrest.org
 
+i18n-util 1.0.1 (https://github.com/salesforce/i18n-util) Copyright (c) 2017, Salesforce.com, Inc. All rights reserved.
+
 ---
 
 This product bundles the following products which are licensed with

http://git-wip-us.apache.org/repos/asf/phoenix/blob/0454e421/phoenix-core/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-core/pom.xml b/phoenix-core/pom.xml
index d331b78..1f54371 100644
--- a/phoenix-core/pom.xml
+++ b/phoenix-core/pom.xml
@@ -471,10 +471,9 @@
       <artifactId>stream</artifactId>
       <version>${stream.version}</version>
     </dependency>
-     <dependency>
+    <dependency>
       <groupId>com.salesforce.i18n</groupId>
       <artifactId>i18n-util</artifactId>
-      <version>1.0.1</version>
     </dependency>
   </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/0454e421/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 1ad8f53..dd816ed 100644
--- a/pom.xml
+++ b/pom.xml
@@ -927,6 +927,11 @@
         <artifactId>stream</artifactId>
         <version>${stream.version}</version>
       </dependency>
+      <dependency>
+        <groupId>com.salesforce.i18n</groupId>
+        <artifactId>i18n-util</artifactId>
+        <version>1.0.1</version>
+      </dependency>
     </dependencies>
   </dependencyManagement>