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 2014/09/20 01:43:28 UTC

[1/5] PHOENIX-180 Use stats to guide query parallelization (Ramkrishna S Vasudevan)

Repository: phoenix
Updated Branches:
  refs/heads/4.0 a18862d06 -> 5cdc938e8


http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/main/java/org/apache/phoenix/schema/stat/StatisticsCollector.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/stat/StatisticsCollector.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/stat/StatisticsCollector.java
new file mode 100644
index 0000000..17b5825
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/stat/StatisticsCollector.java
@@ -0,0 +1,447 @@
+/*
+ * 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.stat;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.Coprocessor;
+import org.apache.hadoop.hbase.CoprocessorEnvironment;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValueUtil;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorException;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.protobuf.ResponseConverter;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.InternalScanner;
+import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
+import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.regionserver.ScanType;
+import org.apache.hadoop.hbase.regionserver.Store;
+import org.apache.hadoop.hbase.regionserver.StoreScanner;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.coprocessor.generated.StatCollectorProtos;
+import org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest;
+import org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse;
+import org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse.Builder;
+import org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectService;
+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.PDataType;
+import org.apache.phoenix.schema.PhoenixArray;
+import org.apache.phoenix.util.TimeKeeper;
+
+import com.google.common.collect.Lists;
+import com.google.protobuf.RpcCallback;
+import com.google.protobuf.RpcController;
+import com.google.protobuf.Service;
+
+/**
+ * An endpoint implementation that allows to collect the stats for a given region and groups the stat per family. This is also an
+ * RegionObserver that collects the information on compaction also. The user would be allowed to invoke this endpoint and thus populate the
+ * Phoenix stats table with the max key, min key and guide posts for the given region. The stats can be consumed by the stats associated
+ * with every PTable and the same can be used to parallelize the queries
+ */
+public class StatisticsCollector extends BaseRegionObserver implements CoprocessorService, Coprocessor,
+        StatisticsTracker, StatCollectService.Interface {
+
+    public static void addToTable(HTableDescriptor desc) throws IOException {
+        desc.addCoprocessor(StatisticsCollector.class.getName());
+    }
+
+    private Map<String, byte[]> minMap = new ConcurrentHashMap<String, byte[]>();
+    private Map<String, byte[]> maxMap = new ConcurrentHashMap<String, byte[]>();
+    private long guidepostDepth;
+    private long byteCount = 0;
+    private Map<String, List<byte[]>> guidePostsMap = new ConcurrentHashMap<String, List<byte[]>>();
+    private Map<ImmutableBytesPtr, Boolean> familyMap = new ConcurrentHashMap<ImmutableBytesPtr, Boolean>();
+    private RegionCoprocessorEnvironment env;
+    protected StatisticsTable stats;
+    // Ensures that either analyze or compaction happens at any point of time.
+    private ReentrantLock lock = new ReentrantLock();
+    private static final Log LOG = LogFactory.getLog(StatisticsCollector.class);
+
+    @Override
+    public void collectStat(RpcController controller, StatCollectRequest request, RpcCallback<StatCollectResponse> done) {
+        HRegion region = env.getRegion();
+        boolean heldLock = false;
+        int count = 0;
+        Builder newBuilder = StatCollectResponse.newBuilder();
+        try {
+            if (lock.tryLock()) {
+                heldLock = true;
+                // Clear all old stats
+                clear();
+                Scan scan = createScan(env.getConfiguration());
+                if (request.hasStartRow()) {
+                    scan.setStartRow(request.getStartRow().toByteArray());
+                }
+                if (request.hasStopRow()) {
+                    scan.setStopRow(request.getStopRow().toByteArray());
+                }
+                RegionScanner scanner = null;
+                try {
+                    scanner = region.getScanner(scan);
+                    count = scanRegion(scanner, count);
+                } catch (IOException e) {
+                    LOG.error(e);
+                    ResponseConverter.setControllerException(controller, e);
+                } finally {
+                    if (scanner != null) {
+                        try {
+                            ArrayList<Mutation> mutations = new ArrayList<Mutation>();
+                            writeStatsToStatsTable(region, scanner, true, mutations, TimeKeeper.SYSTEM.getCurrentTime());
+                            if (LOG.isDebugEnabled()) {
+                                LOG.debug("Committing new stats for the region " + region.getRegionInfo());
+                            }
+                            commitStats(mutations);
+                        } catch (IOException e) {
+                            LOG.error(e);
+                            ResponseConverter.setControllerException(controller, e);
+                        } finally {
+                            clear();
+                        }
+                    }
+                }
+            }
+        } finally {
+            if (heldLock) {
+                lock.unlock();
+            }
+            newBuilder.setRowsScanned(count);
+            StatCollectResponse result = newBuilder.build();
+            done.run(result);
+        }
+    }
+
+    private void writeStatsToStatsTable(final HRegion region,
+            final RegionScanner scanner, boolean delete, List<Mutation> mutations, long currentTime) throws IOException {
+        scanner.close();
+        try {
+            // update the statistics table
+            for (ImmutableBytesPtr fam : familyMap.keySet()) {
+                String tableName = region.getRegionInfo().getTable().getNameAsString();
+                if (delete) {
+                    if(LOG.isDebugEnabled()) {
+                        LOG.debug("Deleting the stats for the region "+region.getRegionInfo());
+                    }
+                    stats.deleteStats(tableName, region.getRegionInfo().getRegionNameAsString(), this,
+                            Bytes.toString(fam.copyBytesIfNecessary()), mutations, currentTime);
+                }
+                if(LOG.isDebugEnabled()) {
+                    LOG.debug("Adding new stats for the region "+region.getRegionInfo());
+                }
+                stats.addStats(tableName, (region.getRegionInfo().getRegionNameAsString()), this,
+                        Bytes.toString(fam.copyBytesIfNecessary()), mutations, currentTime);
+            }
+        } catch (IOException e) {
+            LOG.error("Failed to update statistics table!", e);
+            throw e;
+        }
+    }
+
+    private void commitStats(List<Mutation> mutations) throws IOException {
+        stats.commitStats(mutations);
+    }
+
+    private void deleteStatsFromStatsTable(final HRegion region, List<Mutation> mutations, long currentTime) throws IOException {
+        try {
+            // update the statistics table
+            for (ImmutableBytesPtr fam : familyMap.keySet()) {
+                String tableName = region.getRegionInfo().getTable().getNameAsString();
+                stats.deleteStats(tableName, (region.getRegionInfo().getRegionNameAsString()), this,
+                        Bytes.toString(fam.copyBytesIfNecessary()), mutations, currentTime);
+            }
+        } catch (IOException e) {
+            LOG.error("Failed to delete from statistics table!", e);
+            throw e;
+        }
+    }
+
+    private int scanRegion(RegionScanner scanner, int count) throws IOException {
+        List<Cell> results = new ArrayList<Cell>();
+        boolean hasMore = true;
+        while (hasMore) {
+            // Am getting duplicates here. Need to avoid that
+            hasMore = scanner.next(results);
+            updateStat(results);
+            count += results.size();
+            results.clear();
+            while (!hasMore) {
+                break;
+            }
+        }
+        return count;
+    }
+
+    /**
+     * Update the current statistics based on the lastest batch of key-values from the underlying scanner
+     * 
+     * @param results
+     *            next batch of {@link KeyValue}s
+     */
+    protected void updateStat(final List<Cell> results) {
+        for (Cell c : results) {
+            KeyValue kv = KeyValueUtil.ensureKeyValue(c);
+            updateStatistic(kv);
+        }
+    }
+
+    @Override
+    public Service getService() {
+        return StatCollectorProtos.StatCollectService.newReflectiveService(this);
+    }
+
+    @Override
+    public void start(CoprocessorEnvironment env) throws IOException {
+        if (env instanceof RegionCoprocessorEnvironment) {
+            this.env = (RegionCoprocessorEnvironment)env;
+        } else {
+            throw new CoprocessorException("Must be loaded on a table region!");
+        }
+        HTableDescriptor desc = ((RegionCoprocessorEnvironment)env).getRegion().getTableDesc();
+        // Get the stats table associated with the current table on which the CP is
+        // triggered
+        stats = StatisticsTable.getStatisticsTableForCoprocessor(env, desc.getName());
+        guidepostDepth = env.getConfiguration().getLong(QueryServices.HISTOGRAM_BYTE_DEPTH_CONF_KEY,
+                QueryServicesOptions.DEFAULT_HISTOGRAM_BYTE_DEPTH);
+    }
+
+    @Override
+    public void stop(CoprocessorEnvironment env) throws IOException {
+        if (env instanceof RegionCoprocessorEnvironment) {
+            TableName table = ((RegionCoprocessorEnvironment)env).getRegion().getRegionInfo().getTable();
+            // Close only if the table is system table
+            if(table.getNameAsString().equals(PhoenixDatabaseMetaData.SYSTEM_STATS_NAME)) {
+                stats.close();
+            }
+        }
+    }
+
+    @Override
+    public InternalScanner preCompactScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
+            List<? extends KeyValueScanner> scanners, ScanType scanType, long earliestPutTs, InternalScanner s)
+            throws IOException {
+        InternalScanner internalScan = s;
+        TableName table = c.getEnvironment().getRegion().getRegionInfo().getTable();
+        if (!table.getNameAsString().equals(PhoenixDatabaseMetaData.SYSTEM_STATS_NAME)) {
+            boolean heldLock = false;
+            try {
+                if (lock.tryLock()) {
+                    heldLock = true;
+                    // See if this is for Major compaction
+                    if (scanType.equals(ScanType.COMPACT_DROP_DELETES)) {
+                        // this is the first CP accessed, so we need to just create a major
+                        // compaction scanner, just
+                        // like in the compactor
+                        if (s == null) {
+                            Scan scan = new Scan();
+                            scan.setMaxVersions(store.getFamily().getMaxVersions());
+                            long smallestReadPoint = store.getSmallestReadPoint();
+                            internalScan = new StoreScanner(store, store.getScanInfo(), scan, scanners, scanType,
+                                    smallestReadPoint, earliestPutTs);
+                        }
+                        if (LOG.isDebugEnabled()) {
+                            LOG.debug("Compaction scanner created for stats");
+                        }
+                        InternalScanner scanner = getInternalScanner(c, store, internalScan,
+                                store.getColumnFamilyName());
+                        if (scanner != null) {
+                            internalScan = scanner;
+                        }
+                    }
+                }
+            } finally {
+                if (heldLock) {
+                    lock.unlock();
+                }
+            }
+        }
+        return internalScan;
+    }
+    
+
+    @Override
+    public void postSplit(ObserverContext<RegionCoprocessorEnvironment> ctx, HRegion l, HRegion r) throws IOException {
+        // Invoke collectStat here
+        HRegion region = ctx.getEnvironment().getRegion();
+        TableName table = region.getRegionInfo().getTable();
+        if (!table.getNameAsString().equals(PhoenixDatabaseMetaData.SYSTEM_STATS_NAME)) {
+            if (familyMap != null) {
+                familyMap.clear();
+            }
+            // Create a delete operation on the parent region
+            // Then write the new guide posts for individual regions
+            // TODO : Try making this atomic
+            List<Mutation> mutations = Lists.newArrayListWithExpectedSize(3);
+            long currentTime = TimeKeeper.SYSTEM.getCurrentTime();
+            Configuration conf = ctx.getEnvironment().getConfiguration();
+            if(LOG.isDebugEnabled()) {
+                LOG.debug("Collecting stats for the daughter region "+l.getRegionInfo());
+            }
+            collectStatsForSplitRegions(conf, l, region, true, mutations, currentTime);
+            clear();
+            if(LOG.isDebugEnabled()) {
+                LOG.debug("Collecting stats for the daughter region "+r.getRegionInfo());
+            }
+            collectStatsForSplitRegions(conf, r, region, false, mutations, currentTime);
+            clear();
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("Committing stats for the daughter regions as part of split " + r.getRegionInfo());
+            }
+            commitStats(mutations);
+        }
+    }
+
+    private void collectStatsForSplitRegions(Configuration conf, HRegion daughter, HRegion parent, boolean delete,
+            List<Mutation> mutations, long currentTime) throws IOException {
+        Scan scan = createScan(conf);
+        RegionScanner scanner = null;
+        int count = 0;
+        try {
+            scanner = daughter.getScanner(scan);
+            count = scanRegion(scanner, count);
+        } catch (IOException e) {
+            LOG.error(e);
+            throw e;
+        } finally {
+            if (scanner != null) {
+                try {
+                    if (delete) {
+                        if (LOG.isDebugEnabled()) {
+                            LOG.debug("Deleting the stats for the parent region " + parent.getRegionInfo());
+                        }
+                        deleteStatsFromStatsTable(parent, mutations, currentTime);
+                    }
+                    writeStatsToStatsTable(daughter, scanner, false, mutations, currentTime);
+                } catch (IOException e) {
+                    LOG.error(e);
+                    throw e;
+                }
+            }
+        }
+    }
+
+    private Scan createScan(Configuration conf) {
+        Scan scan = new Scan();
+        scan.setCaching(
+                conf.getInt(QueryServices.SCAN_CACHE_SIZE_ATTRIB, QueryServicesOptions.DEFAULT_SCAN_CACHE_SIZE));
+        // do not cache the blocks here
+        scan.setCacheBlocks(false);
+        return scan;
+    }
+
+    protected InternalScanner getInternalScanner(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
+            InternalScanner internalScan, String family) {
+        return new StatisticsScanner(this, stats, c.getEnvironment().getRegion().getRegionInfo(), internalScan,
+                Bytes.toBytes(family));
+    }
+
+    @Override
+    public void clear() {
+        this.maxMap.clear();
+        this.minMap.clear();
+        this.guidePostsMap.clear();
+        this.familyMap.clear();
+    }
+
+    @Override
+    public void updateStatistic(KeyValue kv) {
+        byte[] cf = kv.getFamily();
+        familyMap.put(new ImmutableBytesPtr(cf), true);
+        
+        String fam = Bytes.toString(cf);
+        byte[] row = new ImmutableBytesPtr(kv.getRowArray(), kv.getRowOffset(), kv.getRowLength())
+                .copyBytesIfNecessary();
+        if (!minMap.containsKey(fam) && !maxMap.containsKey(fam)) {
+            minMap.put(fam, row);
+            // Ideally the max key also should be added in this case
+            maxMap.put(fam, row);
+        } else {
+            if (Bytes.compareTo(kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(), minMap.get(fam), 0,
+                    minMap.get(fam).length) < 0) {
+                minMap.put(fam, row);
+            }
+            if (Bytes.compareTo(kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(), maxMap.get(fam), 0,
+                    maxMap.get(fam).length) > 0) {
+                maxMap.put(fam, row);
+            }
+        }
+        byteCount += kv.getLength();
+        // TODO : This can be moved to an interface so that we could collect guide posts in different ways
+        if (byteCount >= guidepostDepth) {
+            if (guidePostsMap.get(fam) != null) {
+                guidePostsMap.get(fam).add(
+                        row);
+            } else {
+                List<byte[]> guidePosts = new ArrayList<byte[]>();
+                guidePosts.add(row);
+                guidePostsMap.put(fam, guidePosts);
+            }
+            // reset the count for the next key
+            byteCount = 0;
+        }
+    }
+
+    @Override
+    public byte[] getMaxKey(String fam) {
+        if (maxMap.get(fam) != null) { return maxMap.get(fam); }
+        return null;
+    }
+
+    @Override
+    public byte[] getMinKey(String fam) {
+        if (minMap.get(fam) != null) { return minMap.get(fam); }
+        return null;
+    }
+
+    @Override
+    public byte[] getGuidePosts(String fam) {
+        if (!guidePostsMap.isEmpty()) {
+            List<byte[]> guidePosts = guidePostsMap.get(fam);
+            if (guidePosts != null) {
+                byte[][] array = new byte[guidePosts.size()][];
+                int i = 0;
+                for (byte[] element : guidePosts) {
+                    array[i] = element;
+                    i++;
+                }
+                PhoenixArray phoenixArray = new PhoenixArray(PDataType.VARBINARY, array);
+                return PDataType.VARBINARY_ARRAY.toBytes(phoenixArray);
+            }
+        }
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/main/java/org/apache/phoenix/schema/stat/StatisticsScanner.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/stat/StatisticsScanner.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/stat/StatisticsScanner.java
new file mode 100644
index 0000000..09174b2
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/stat/StatisticsScanner.java
@@ -0,0 +1,117 @@
+/*
+ * 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.stat;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValueUtil;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.regionserver.InternalScanner;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.TimeKeeper;
+
+/**
+ * The scanner that does the scanning to collect the stats during major compaction.{@link StatisticsCollector}
+ */
+public class StatisticsScanner implements InternalScanner {
+    private static final Log LOG = LogFactory.getLog(StatisticsScanner.class);
+    private InternalScanner delegate;
+    private StatisticsTable stats;
+    private HRegionInfo region;
+    private StatisticsTracker tracker;
+    private byte[] family;
+
+    public StatisticsScanner(StatisticsTracker tracker, StatisticsTable stats, HRegionInfo region,
+            InternalScanner delegate, byte[] family) {
+        // should there be only one tracker?
+        this.tracker = tracker;
+        this.stats = stats;
+        this.delegate = delegate;
+        this.region = region;
+        this.family = family;
+        this.tracker.clear();
+    }
+
+    @Override
+    public boolean next(List<Cell> result) throws IOException {
+        boolean ret = delegate.next(result);
+        updateStat(result);
+        return ret;
+    }
+
+    @Override
+    public boolean next(List<Cell> result, int limit) throws IOException {
+        boolean ret = delegate.next(result, limit);
+        updateStat(result);
+        return ret;
+    }
+
+    /**
+     * Update the current statistics based on the lastest batch of key-values from the underlying scanner
+     * 
+     * @param results
+     *            next batch of {@link KeyValue}s
+     */
+    protected void updateStat(final List<Cell> results) {
+        for (Cell c : results) {
+            KeyValue kv = KeyValueUtil.ensureKeyValue(c);
+            if (c.getTypeByte() == KeyValue.Type.Put.getCode()) {
+                tracker.updateStatistic(kv);
+            }
+        }
+    }
+
+    public void close() throws IOException {
+        IOException toThrow = null;
+        try {
+            // update the statistics table
+            // Just verify if this if fine
+            String tableName = SchemaUtil.getTableNameFromFullName(region.getTable().getNameAsString());
+            ArrayList<Mutation> mutations = new ArrayList<Mutation>();
+            long currentTime = TimeKeeper.SYSTEM.getCurrentTime();
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("Deleting the stats for the region " + region.getRegionNameAsString()
+                        + " as part of major compaction");
+            }
+            stats.deleteStats(tableName, region.getRegionNameAsString(), this.tracker, Bytes.toString(family),
+                    mutations, currentTime);
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("Adding new stats for the region " + region.getRegionNameAsString()
+                        + " as part of major compaction");
+            }
+            stats.addStats(tableName, region.getRegionNameAsString(), this.tracker, Bytes.toString(family), mutations,
+                    currentTime);
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("Committing new stats for the region " + region.getRegionNameAsString()
+                        + " as part of major compaction");
+            }
+            stats.commitStats(mutations);
+        } catch (IOException e) {
+            LOG.error("Failed to update statistics table!", e);
+            toThrow = e;
+        }
+        // close the delegate scanner
+        try {
+            delegate.close();
+        } catch (IOException e) {
+            LOG.error("Error while closing the scanner");
+            // TODO : We should throw the exception
+            /*if (toThrow == null) { throw e; }
+            throw MultipleIOException.createIOException(Lists.newArrayList(toThrow, e));*/
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/main/java/org/apache/phoenix/schema/stat/StatisticsTable.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/stat/StatisticsTable.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/stat/StatisticsTable.java
new file mode 100644
index 0000000..fcbbee9
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/stat/StatisticsTable.java
@@ -0,0 +1,168 @@
+/*
+ * 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.stat;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.sql.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CoprocessorEnvironment;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.HTablePool;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.PDataType;
+
+/**
+ * Wrapper to access the statistics table SYSTEM.STATS using the HTable.
+ */
+@SuppressWarnings("deprecation")
+public class StatisticsTable implements Closeable {
+    /** Map of the currently open statistics tables */
+    private static final Map<String, StatisticsTable> tableMap = new HashMap<String, StatisticsTable>();
+    /**
+     * @param env
+     *            Environment wherein the coprocessor is attempting to update the stats table.
+     * @param primaryTableName
+     *            name of the primary table on which we should collect stats
+     * @return the {@link StatisticsTable} for the given primary table.
+     * @throws IOException
+     *             if the table cannot be created due to an underlying HTable creation error
+     */
+    public synchronized static StatisticsTable getStatisticsTableForCoprocessor(CoprocessorEnvironment env,
+            byte[] primaryTableName) throws IOException {
+        StatisticsTable table = tableMap.get(primaryTableName);
+        if (table == null) {
+            // Map the statics table and the table with which the statistics is
+            // associated. This is a workaround
+            HTablePool pool = new HTablePool(env.getConfiguration(), 1);
+            HTableInterface hTable = pool.getTable(PhoenixDatabaseMetaData.SYSTEM_STATS_NAME);
+            table = new StatisticsTable(hTable, primaryTableName);
+            tableMap.put(Bytes.toString(primaryTableName), table);
+        }
+        return table;
+    }
+
+    private final HTableInterface statisticsTable;
+    private final byte[] sourceTableName;
+
+    private StatisticsTable(HTableInterface statsTable, byte[] sourceTableName) {
+        this.statisticsTable = statsTable;
+        this.sourceTableName = sourceTableName;
+    }
+
+    public StatisticsTable(Configuration conf, HTableDescriptor source) throws IOException {
+        this(new HTable(conf, PhoenixDatabaseMetaData.SYSTEM_STATS_NAME), source.getName());
+    }
+
+    /**
+     * Close the connection to the table
+     */
+    @Override
+    public void close() throws IOException {
+        statisticsTable.close();
+    }
+
+    /**
+     * Update a list of statistics for a given region.  If the ANALYZE <tablename> query is issued
+     * then we use Upsert queries to update the table
+     * If the region gets splitted or the major compaction happens we update using HTable.put()
+     * @param tablekey - The table name
+     * @param schemaName - the schema name associated with the table          
+     * @param region name -  the region of the table for which the stats are collected
+     * @param tracker - the statistics tracker
+     * @param fam -  the family for which the stats is getting collected.
+     * @param split - if the updation is caused due to a split
+     * @param mutations - list of mutations that collects all the mutations to commit in a batch
+     * @param currentTime -  the current time
+     * @throws IOException
+     *             if we fail to do any of the puts. Any single failure will prevent any future attempts for the remaining list of stats to
+     *             update
+     */
+    public void addStats(String tableName, String regionName, StatisticsTracker tracker, String fam,
+            List<Mutation> mutations, long currentTime) throws IOException {
+        if (tracker == null) { return; }
+
+        // Add the timestamp header
+        formLastUpdatedStatsMutation(tableName, currentTime, mutations);
+
+        byte[] prefix = StatisticsUtils.getRowKey(PDataType.VARCHAR.toBytes(tableName), PDataType.VARCHAR.toBytes(fam),
+                PDataType.VARCHAR.toBytes(regionName));
+        formStatsUpdateMutation(tracker, fam, mutations, currentTime, prefix);
+    }
+
+    public void commitStats(List<Mutation> mutations) throws IOException {
+        Object[] res = new Object[mutations.size()];
+        try {
+            statisticsTable.batch(mutations, res);
+        } catch (InterruptedException e) {
+            throw new IOException("Exception while adding deletes and puts");
+        }
+    }
+
+    private void formStatsUpdateMutation(StatisticsTracker tracker, String fam, List<Mutation> mutations,
+            long currentTime, byte[] prefix) {
+        Put put = new Put(prefix, currentTime);
+        if (tracker.getGuidePosts(fam) != null) {
+            put.add(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, PhoenixDatabaseMetaData.GUIDE_POSTS_BYTES,
+                    currentTime, (tracker.getGuidePosts(fam)));
+        }
+        put.add(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, PhoenixDatabaseMetaData.MIN_KEY_BYTES,
+                currentTime, PDataType.VARBINARY.toBytes(tracker.getMinKey(fam)));
+        put.add(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, PhoenixDatabaseMetaData.MAX_KEY_BYTES,
+                currentTime, PDataType.VARBINARY.toBytes(tracker.getMaxKey(fam)));
+        mutations.add(put);
+    }
+
+    private void formLastUpdatedStatsMutation(String tableName, long currentTime, List<Mutation> mutations) throws IOException {
+        byte[] prefix = StatisticsUtils.getRowKeyForTSUpdate(PDataType.VARCHAR.toBytes(tableName));
+        Put put = new Put(prefix);
+        put.add(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, PhoenixDatabaseMetaData.LAST_STATS_UPDATE_TIME_BYTES, currentTime,
+                PDataType.DATE.toBytes(new Date(currentTime)));
+        mutations.add(put);
+    }
+    
+    public void deleteStats(String tableName, String regionName, StatisticsTracker tracker, String fam,
+            List<Mutation> mutations, long currentTime)
+            throws IOException {
+        byte[] prefix = StatisticsUtils.getRowKey(PDataType.VARCHAR.toBytes(tableName), PDataType.VARCHAR.toBytes(fam),
+                PDataType.VARCHAR.toBytes(regionName));
+        mutations.add(new Delete(prefix, currentTime - 1));
+    }
+
+    /**
+     * @return the underlying {@link HTableInterface} to which this table is writing
+     */
+    HTableInterface getUnderlyingTable() {
+        return statisticsTable;
+    }
+
+    byte[] getSourceTableName() {
+        return this.sourceTableName;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/main/java/org/apache/phoenix/schema/stat/StatisticsTracker.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/stat/StatisticsTracker.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/stat/StatisticsTracker.java
new file mode 100644
index 0000000..e1754f3
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/stat/StatisticsTracker.java
@@ -0,0 +1,62 @@
+/*
+ * 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.stat;
+
+import org.apache.hadoop.hbase.KeyValue;
+
+/**
+ * Track a statistic for the column on a given region
+ */
+public interface StatisticsTracker {
+
+    /**
+     * Reset the statistic after the completion of the compaction
+     */
+    public void clear();
+
+    /**
+     * Update the current statistics with the next {@link KeyValue} to be written
+     * 
+     * @param kv
+     *            next {@link KeyValue} to be written.
+     */
+    public void updateStatistic(KeyValue kv);
+
+    /**
+     * Return the max key of the family
+     * @param fam
+     * @return
+     */
+    public byte[] getMaxKey(String fam);
+
+    /**
+     * Return the min key of the family
+     * 
+     * @param fam
+     * @return
+     */
+    public byte[] getMinKey(String fam);
+
+    /**
+     * Return the guide posts of the family
+     * 
+     * @param fam
+     * @return
+     */
+    public byte[] getGuidePosts(String fam);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/main/java/org/apache/phoenix/schema/stat/StatisticsUtils.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/stat/StatisticsUtils.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/stat/StatisticsUtils.java
new file mode 100644
index 0000000..7cb3a38
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/stat/StatisticsUtils.java
@@ -0,0 +1,80 @@
+/*
+ * 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.stat;
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.util.TrustedByteArrayOutputStream;
+/**
+ * Simple utility class for managing multiple key parts of the statistic
+ */
+public class StatisticsUtils {
+
+    private StatisticsUtils() {
+        // private ctor for utility classes
+    }
+
+    /** Number of parts in our complex key */
+    protected static final int NUM_KEY_PARTS = 3;
+
+    public static byte[] getRowKey(byte[] table, byte[] fam, byte[] region) throws IOException {
+        // always starts with the source table
+        TrustedByteArrayOutputStream os = new TrustedByteArrayOutputStream(table.length + region.length
+                + fam.length + (NUM_KEY_PARTS - 1));
+        os.write(table);
+        os.write(QueryConstants.SEPARATOR_BYTE_ARRAY);
+        os.write(fam);
+        os.write(QueryConstants.SEPARATOR_BYTE_ARRAY);
+        os.write(region);
+        os.close();
+        return os.getBuffer();
+    }
+    
+    public static byte[] getRowKeyForTSUpdate(byte[] table) throws IOException {
+        // always starts with the source table
+        TrustedByteArrayOutputStream os = new TrustedByteArrayOutputStream(table.length);
+        os.write(table);
+        os.close();
+        return os.getBuffer();
+    }
+
+    public static byte[] getCFFromRowKey(byte[] table, byte[] row, int rowOffset, int rowLength) {
+        // Move over the the sepeartor byte that would be written after the table name
+        int startOff = Bytes.indexOf(row, table) + (table.length) + 1;
+        int endOff = startOff;
+        while (endOff < rowLength) {
+            // Check for next seperator byte
+            if (row[endOff] != QueryConstants.SEPARATOR_BYTE) {
+                endOff++;
+            } else {
+                break;
+            }
+        }
+        int cfLength = endOff - startOff;
+        byte[] cf = new byte[cfLength];
+        System.arraycopy(row, startOff, cf, 0, cfLength);
+        return cf;
+    }
+
+    public static byte[] copyRow(KeyValue kv) {
+        return Arrays.copyOfRange(kv.getRowArray(), kv.getRowOffset(), kv.getRowOffset() + kv.getRowLength());
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/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 e17e9bf..37285f6 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
@@ -19,7 +19,6 @@ package org.apache.phoenix.query;
 
 import static org.apache.phoenix.util.PhoenixRuntime.CURRENT_SCN_ATTRIB;
 import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL;
-import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR;
 import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL_TERMINATOR;
 import static org.apache.phoenix.util.PhoenixRuntime.PHOENIX_TEST_DRIVER_URL_PARAM;
 import static org.apache.phoenix.util.TestUtil.ATABLE_NAME;
@@ -51,7 +50,6 @@ import static org.apache.phoenix.util.TestUtil.JOIN_ITEM_TABLE_FULL_NAME;
 import static org.apache.phoenix.util.TestUtil.JOIN_ORDER_TABLE_FULL_NAME;
 import static org.apache.phoenix.util.TestUtil.JOIN_SUPPLIER_TABLE_FULL_NAME;
 import static org.apache.phoenix.util.TestUtil.KEYONLY_NAME;
-import static org.apache.phoenix.util.TestUtil.LOCALHOST;
 import static org.apache.phoenix.util.TestUtil.MDTEST_NAME;
 import static org.apache.phoenix.util.TestUtil.MILLIS_IN_DAY;
 import static org.apache.phoenix.util.TestUtil.MULTI_CF_NAME;
@@ -125,7 +123,12 @@ import org.apache.phoenix.schema.NewerTableAlreadyExistsException;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.TableAlreadyExistsException;
 import org.apache.phoenix.schema.TableNotFoundException;
-import org.apache.phoenix.util.*;
+import org.apache.phoenix.util.ConfigUtil;
+import org.apache.phoenix.util.PhoenixRuntime;
+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.junit.Assert;
 
 import com.google.common.collect.ImmutableMap;
@@ -548,6 +551,7 @@ public abstract class BaseTest {
         conf.setInt("hbase.hlog.asyncer.number", 2);
         conf.setInt("hbase.assignment.zkevent.workers", 5);
         conf.setInt("hbase.assignment.threads.max", 5);
+        conf.setInt(QueryServices.HISTOGRAM_BYTE_DEPTH_CONF_KEY, 20);
         return conf;
     }
 
@@ -1225,13 +1229,12 @@ public abstract class BaseTest {
         try {
             HTableDescriptor[] tables = admin.listTables();
             for (HTableDescriptor table : tables) {
-                boolean isCatalogTable = (Bytes.compareTo(table.getName(), PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES) == 0);
-                boolean isSequenceTable = (Bytes.compareTo(table.getName(), PhoenixDatabaseMetaData.SEQUENCE_TABLE_NAME_BYTES) == 0);
-                if (!isCatalogTable && !isSequenceTable) {
+                String schemaName = SchemaUtil.getSchemaNameFromFullName(table.getName());
+                if (!QueryConstants.SYSTEM_SCHEMA_NAME.equals(schemaName)) {
                     admin.disableTable(table.getName());
                     admin.deleteTable(table.getName());
                 }
-            }    
+            }
         } finally {
             admin.close();
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java b/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java
index 47f5b1b..2e3c8f7 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java
@@ -32,7 +32,7 @@ import org.apache.phoenix.util.ReadOnlyProps;
  */
 public final class QueryServicesTestImpl extends BaseQueryServicesImpl {
 
-    private static final int DEFAULT_THREAD_POOL_SIZE = 8;
+    private static final int DEFAULT_THREAD_POOL_SIZE = 20;
     private static final int DEFAULT_QUEUE_SIZE = 0;
     // TODO: setting this down to 5mb causes insufficient memory exceptions. Need to investigate why
     private static final int DEFAULT_MAX_MEMORY_PERC = 30; // 30% of heap
@@ -42,8 +42,6 @@ public final class QueryServicesTestImpl extends BaseQueryServicesImpl {
     private static final int DEFAULT_MAX_TENANT_MEMORY_PERC = 100;
     private static final int DEFAULT_MAX_SERVER_CACHE_TIME_TO_LIVE_MS = 60000 * 60; // 1HR (to prevent age-out of hash cache during debugging)
     private static final long DEFAULT_MAX_HASH_CACHE_SIZE = 1024*1024*10;  // 10 Mb
-    private static final int DEFAULT_TARGET_QUERY_CONCURRENCY = 4;
-    private static final int DEFAULT_MAX_QUERY_CONCURRENCY = 8;
     private static final boolean DEFAULT_DROP_METADATA = false;
     
     private static final int DEFAULT_MASTER_INFO_PORT = -1;
@@ -69,8 +67,6 @@ public final class QueryServicesTestImpl extends BaseQueryServicesImpl {
                 .setMaxMemoryWaitMs(DEFAULT_MAX_MEMORY_WAIT_MS)
                 .setMaxTenantMemoryPerc(DEFAULT_MAX_TENANT_MEMORY_PERC)
                 .setMaxServerCacheSize(DEFAULT_MAX_HASH_CACHE_SIZE)
-                .setTargetQueryConcurrency(DEFAULT_TARGET_QUERY_CONCURRENCY)
-                .setMaxQueryConcurrency(DEFAULT_MAX_QUERY_CONCURRENCY)
                 .setRowKeyOrderSaltedTable(true)
                 .setMaxServerCacheTTLMs(DEFAULT_MAX_SERVER_CACHE_TIME_TO_LIVE_MS)
                 .setMasterInfoPort(DEFAULT_MASTER_INFO_PORT)

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-protocol/src/main/MetaDataService.proto
----------------------------------------------------------------------
diff --git a/phoenix-protocol/src/main/MetaDataService.proto b/phoenix-protocol/src/main/MetaDataService.proto
index a766674..9174b4d 100644
--- a/phoenix-protocol/src/main/MetaDataService.proto
+++ b/phoenix-protocol/src/main/MetaDataService.proto
@@ -92,6 +92,16 @@ message GetVersionResponse {
   required int64 version = 1;
 }
 
+message ClearCacheForTableRequest {
+  required bytes tenantId = 1;
+  required bytes schemaName  = 2;
+  required bytes tableName = 3;
+  required int64 clientTimestamp = 4;
+}
+
+message ClearCacheForTableResponse {
+}
+
 service MetaDataService {
   rpc getTable(GetTableRequest)
     returns (MetaDataResponse);
@@ -115,5 +125,8 @@ service MetaDataService {
     returns (ClearCacheResponse);
     
    rpc getVersion(GetVersionRequest)
-    returns (GetVersionResponse);           
+    returns (GetVersionResponse);
+   
+   rpc clearCacheForTable(ClearCacheForTableRequest)
+    returns (ClearCacheForTableResponse);
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-protocol/src/main/PTable.proto
----------------------------------------------------------------------
diff --git a/phoenix-protocol/src/main/PTable.proto b/phoenix-protocol/src/main/PTable.proto
index 20c63e1..3b5f5cf 100644
--- a/phoenix-protocol/src/main/PTable.proto
+++ b/phoenix-protocol/src/main/PTable.proto
@@ -72,6 +72,6 @@ message PTable {
   optional bytes viewStatement = 18;
   repeated bytes physicalNames = 19;
   optional bytes tenantId = 20;
-  optional int32 viewIndexId = 21; 
+  optional int32 viewIndexId = 21;
   optional bytes indexType = 22;
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-protocol/src/main/StatisticsCollect.proto
----------------------------------------------------------------------
diff --git a/phoenix-protocol/src/main/StatisticsCollect.proto b/phoenix-protocol/src/main/StatisticsCollect.proto
new file mode 100644
index 0000000..c80a756
--- /dev/null
+++ b/phoenix-protocol/src/main/StatisticsCollect.proto
@@ -0,0 +1,20 @@
+option java_package = "org.apache.phoenix.coprocessor.generated";
+option java_outer_classname = "StatCollectorProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+
+message StatCollectRequest {
+  optional bytes startRow = 1;
+  optional bytes stopRow = 2;
+}
+
+message StatCollectResponse {
+  required uint64 rowsScanned = 1;
+}
+
+service StatCollectService {
+  rpc collectStat(StatCollectRequest)
+    returns (StatCollectResponse);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 4ffb4ba..f97473f 100644
--- a/pom.xml
+++ b/pom.xml
@@ -115,7 +115,7 @@
     
     <!-- Plugin options -->
     <numForkedUT>3</numForkedUT>
-    <numForkedIT>7</numForkedIT>
+    <numForkedIT>5</numForkedIT>
     
     <!-- Set default encoding so multi-byte tests work correctly on the Mac -->
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>


[2/5] PHOENIX-180 Use stats to guide query parallelization (Ramkrishna S Vasudevan)

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/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 7e1e0a5..987d200 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
@@ -79,6 +79,8 @@ import org.apache.phoenix.coprocessor.ServerCachingEndpointImpl;
 import org.apache.phoenix.coprocessor.UngroupedAggregateRegionObserver;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.AddColumnRequest;
+import org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest;
+import org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheRequest;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheResponse;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest;
@@ -90,6 +92,9 @@ import org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionRespons
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataResponse;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataService;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.UpdateIndexStateRequest;
+import org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest;
+import org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse;
+import org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectService;
 import org.apache.phoenix.exception.PhoenixIOException;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
@@ -123,6 +128,7 @@ import org.apache.phoenix.schema.Sequence;
 import org.apache.phoenix.schema.SequenceKey;
 import org.apache.phoenix.schema.TableAlreadyExistsException;
 import org.apache.phoenix.schema.TableNotFoundException;
+import org.apache.phoenix.schema.stat.StatisticsCollector;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.Closeables;
 import org.apache.phoenix.util.ConfigUtil;
@@ -141,6 +147,7 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 import com.google.protobuf.HBaseZeroCopyByteString;
+import com.google.protobuf.ServiceException;
 
 
 public class ConnectionQueryServicesImpl extends DelegateQueryServices implements ConnectionQueryServices {
@@ -153,7 +160,6 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     private final ReadOnlyProps props;
     private final String userName;
     private final ConcurrentHashMap<ImmutableBytesWritable,ConnectionQueryServices> childServices;
-    private final StatsManager statsManager;
     
     // Cache the latest meta data here for future connections
     // writes guarded by "latestMetaDataLock"
@@ -211,10 +217,6 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         // TODO: should we track connection wide memory usage or just org-wide usage?
         // If connection-wide, create a MemoryManager here, otherwise just use the one from the delegate
         this.childServices = new ConcurrentHashMap<ImmutableBytesWritable,ConnectionQueryServices>(INITIAL_CHILD_SERVICES_CAPACITY);
-        int statsUpdateFrequencyMs = this.getProps().getInt(QueryServices.STATS_UPDATE_FREQ_MS_ATTRIB, QueryServicesOptions.DEFAULT_STATS_UPDATE_FREQ_MS);
-        int maxStatsAgeMs = this.getProps().getInt(QueryServices.MAX_STATS_AGE_MS_ATTRIB, QueryServicesOptions.DEFAULT_MAX_STATS_AGE_MS);
-        this.statsManager = new StatsManagerImpl(this, statsUpdateFrequencyMs, maxStatsAgeMs);
-        
         // find the HBase version and use that to determine the KeyValueBuilder that should be used
         String hbaseVersion = VersionInfo.getVersion();
         this.kvBuilder = KeyValueBuilder.get(hbaseVersion);
@@ -242,11 +244,6 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     }
 
     @Override
-    public StatsManager getStatsManager() {
-        return this.statsManager;
-    }
-    
-    @Override
     public HTableInterface getTable(byte[] tableName) throws SQLException {
         try {
             return HBaseFactoryProvider.getHTableFactory().getTable(tableName, connection, getExecutor());
@@ -307,42 +304,29 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 sqlE = e;
             } finally {
                 try {
-                    // Clear any client-side caches.  
-                    statsManager.clearStats();
-                } catch (SQLException e) {
+                    childServices.clear();
+                    synchronized (latestMetaDataLock) {
+                        latestMetaData = null;
+                        latestMetaDataLock.notifyAll();
+                    }
+                    if (connection != null) connection.close();
+                } catch (IOException e) {
                     if (sqlE == null) {
-                        sqlE = e;
+                        sqlE = ServerUtil.parseServerException(e);
                     } else {
-                        sqlE.setNextException(e);
+                        sqlE.setNextException(ServerUtil.parseServerException(e));
                     }
                 } finally {
                     try {
-                        childServices.clear();
-                        synchronized (latestMetaDataLock) {
-                            latestMetaData = null;
-                            latestMetaDataLock.notifyAll();
-                        }
-                        if (connection != null) connection.close();
-                    } catch (IOException e) {
+                        super.close();
+                    } catch (SQLException e) {
                         if (sqlE == null) {
-                            sqlE = ServerUtil.parseServerException(e);
+                            sqlE = e;
                         } else {
-                            sqlE.setNextException(ServerUtil.parseServerException(e));
+                            sqlE.setNextException(e);
                         }
                     } finally {
-                        try {
-                            super.close();
-                        } catch (SQLException e) {
-                            if (sqlE == null) {
-                                sqlE = e;
-                            } else {
-                                sqlE.setNextException(e);
-                            }
-                        } finally {
-                            if (sqlE != null) {
-                                throw sqlE;
-                            }
-                        }
+                        if (sqlE != null) { throw sqlE; }
                     }
                 }
             }
@@ -615,7 +599,10 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             if (!descriptor.hasCoprocessor(ServerCachingEndpointImpl.class.getName())) {
                 descriptor.addCoprocessor(ServerCachingEndpointImpl.class.getName(), null, 1, null);
             }
-            
+
+            if (!descriptor.hasCoprocessor(StatisticsCollector.class.getName())) {
+                descriptor.addCoprocessor(StatisticsCollector.class.getName(), null, 1, null);
+            }
             // TODO: better encapsulation for this
             // Since indexes can't have indexes, don't install our indexing coprocessor for indexes. Also,
             // don't install on the metadata table until we fix the TODO there.
@@ -1531,19 +1518,12 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                             }
                             try {
                                 metaConnection.createStatement().executeUpdate(QueryConstants.CREATE_SEQUENCE_METADATA);
+                                
+                                // TODO : Get this from a configuration
+                                metaConnection.createStatement().executeUpdate(
+                                        QueryConstants.CREATE_STATS_TABLE_METADATA);
                             } catch (NewerTableAlreadyExistsException ignore) {
-                                // Ignore, as this will happen if the SYSTEM.SEQUENCE already exists at this fixed timestamp.
-                                // A TableAlreadyExistsException is not thrown, since the table only exists *after* this fixed timestamp.
                             } catch (TableAlreadyExistsException ignore) {
-                                // This will occur if we have an older SYSTEM.SEQUENCE, so we need to update it to include
-                                // any new columns we've added.
-                                String newColumns =
-                                        MIN_VALUE + " " + PDataType.LONG.getSqlTypeName() + ", " 
-                                                + MAX_VALUE + " " + PDataType.LONG.getSqlTypeName() + ", " 
-                                                + CYCLE_FLAG + " " + PDataType.BOOLEAN.getSqlTypeName() + ", " 
-                                                + LIMIT_REACHED_FLAG + " " + PDataType.BOOLEAN.getSqlTypeName();
-                                metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SEQUENCE_TABLE_NAME, 
-                                    MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP, newColumns); 
                             }
                         } catch (Exception e) {
                             if (e instanceof SQLException) {
@@ -1873,6 +1853,92 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
     }
 
+    @Override
+    public long updateStatistics(final KeyRange keyRange, final byte[] tableName) throws SQLException {
+        HTableInterface ht = null;
+        try {
+            ht = this.getTable(tableName);
+            Batch.Call<StatCollectService, StatCollectResponse> callable = new Batch.Call<StatCollectService, StatCollectResponse>() {
+                ServerRpcController controller = new ServerRpcController();
+                BlockingRpcCallback<StatCollectResponse> rpcCallback = new BlockingRpcCallback<StatCollectResponse>();
+
+                @Override
+                public StatCollectResponse call(StatCollectService service) throws IOException {
+                    StatCollectRequest.Builder builder = StatCollectRequest.newBuilder();
+                    builder.setStartRow(HBaseZeroCopyByteString.wrap(keyRange.getLowerRange()));
+                    builder.setStopRow(HBaseZeroCopyByteString.wrap(keyRange.getUpperRange()));
+                    service.collectStat(controller, builder.build(), rpcCallback);
+                    if (controller.getFailedOn() != null) { throw controller.getFailedOn(); }
+                    return rpcCallback.get();
+                }
+            };
+            Map<byte[], StatCollectResponse> result = ht.coprocessorService(StatCollectService.class,
+                    keyRange.getLowerRange(), keyRange.getUpperRange(), callable);
+            StatCollectResponse next = result.values().iterator().next();
+            return next.getRowsScanned();
+        } catch (ServiceException e) {
+            throw new SQLException("Unable to update the statistics for the table " + tableName, e);
+        } catch (TableNotFoundException e) {
+            throw new SQLException("Unable to update the statistics for the table " + tableName, e);
+        } catch (Throwable e) {
+            throw new SQLException("Unable to update the statistics for the table " + tableName, e);
+        } finally {
+            if (ht != null) {
+                try {
+                    ht.close();
+                } catch (IOException e) {
+                    throw new SQLException("Unable to close the table " + tableName + " after collecting stats", e);
+                }
+            }
+        }
+    }
+
+    @Override
+    public void clearCacheForTable(final byte[] tenantId, final byte[] schemaName, final byte[] tableName,
+            final long clientTS) throws SQLException {
+        // clear the meta data cache for the table here
+        try {
+            SQLException sqlE = null;
+            HTableInterface htable = this.getTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
+            try {
+                htable.coprocessorService(MetaDataService.class, HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW,
+                        new Batch.Call<MetaDataService, ClearCacheForTableResponse>() {
+                            @Override
+                            public ClearCacheForTableResponse call(MetaDataService instance) throws IOException {
+                                ServerRpcController controller = new ServerRpcController();
+                                BlockingRpcCallback<ClearCacheForTableResponse> rpcCallback = new BlockingRpcCallback<ClearCacheForTableResponse>();
+                                ClearCacheForTableRequest.Builder builder = ClearCacheForTableRequest.newBuilder();
+                                builder.setTenantId(HBaseZeroCopyByteString.wrap(tenantId));
+                                builder.setTableName(HBaseZeroCopyByteString.wrap(tableName));
+                                builder.setSchemaName(HBaseZeroCopyByteString.wrap(schemaName));
+                                builder.setClientTimestamp(clientTS);
+                                instance.clearCacheForTable(controller, builder.build(), rpcCallback);
+                                if (controller.getFailedOn() != null) { throw controller.getFailedOn(); }
+                                return rpcCallback.get();
+                            }
+                        });
+            } catch (IOException e) {
+                throw ServerUtil.parseServerException(e);
+            } catch (Throwable e) {
+                sqlE = new SQLException(e);
+            } finally {
+                try {
+                    htable.close();
+                } catch (IOException e) {
+                    if (sqlE == null) {
+                        sqlE = ServerUtil.parseServerException(e);
+                    } else {
+                        sqlE.setNextException(ServerUtil.parseServerException(e));
+                    }
+                } finally {
+                    if (sqlE != null) { throw sqlE; }
+                }
+            }
+        } catch (Exception e) {
+            throw new SQLException(ServerUtil.parseServerException(e));
+        }
+    }
+
     @SuppressWarnings("deprecation")
     @Override
     public void returnSequences(List<SequenceKey> keys, long timestamp, SQLException[] exceptions) throws SQLException {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
index 61c2ef8..9fa415c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
@@ -68,7 +68,6 @@ import org.apache.phoenix.schema.SequenceKey;
 import org.apache.phoenix.schema.SequenceNotFoundException;
 import org.apache.phoenix.schema.TableAlreadyExistsException;
 import org.apache.phoenix.schema.TableNotFoundException;
-import org.apache.phoenix.schema.TableRef;
 import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.PropertiesUtil;
@@ -121,30 +120,6 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
     }
 
     @Override
-    public StatsManager getStatsManager() {
-        return new StatsManager() {
-
-            @Override
-            public byte[] getMinKey(TableRef table) {
-                return HConstants.EMPTY_START_ROW;
-            }
-
-            @Override
-            public byte[] getMaxKey(TableRef table) {
-                return HConstants.EMPTY_END_ROW;
-            }
-
-            @Override
-            public void updateStats(TableRef table) throws SQLException {
-            }
-
-            @Override
-            public void clearStats() throws SQLException {
-            }
-        };
-    }
-
-    @Override
     public List<HRegionLocation> getAllTableRegions(byte[] tableName) throws SQLException {
         return Collections.singletonList(new HRegionLocation(
             new HRegionInfo(TableName.valueOf(tableName), HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW),
@@ -214,6 +189,15 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
         return new MetaDataMutationResult(MutationCode.TABLE_ALREADY_EXISTS, 0, null);
     }
 
+    @Override
+    public long updateStatistics(KeyRange keyRange, byte[] tableName) throws SQLException {
+        // Noop
+        return 0;
+    }
+    
+    @Override
+    public void clearCacheForTable(byte[] tenantId, byte[] schemaName, byte[] tableName, long clientTS)
+            throws SQLException {}
     // TODO: share this with ConnectionQueryServicesImpl
     @Override
     public void init(String url, Properties props) throws SQLException {
@@ -249,6 +233,15 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
                     // Ignore, as this will happen if the SYSTEM.SEQUENCE already exists at this fixed timestamp.
                     // A TableAlreadyExistsException is not thrown, since the table only exists *after* this fixed timestamp.
                 }
+                try {
+                    // TODO : Get this from a configuration
+                    metaConnection.createStatement().executeUpdate(QueryConstants.CREATE_STATS_TABLE_METADATA);
+                } catch (NewerTableAlreadyExistsException ignore) {
+                    // Ignore, as this will happen if the SYSTEM.SEQUENCE already exists at this fixed
+                    // timestamp.
+                    // A TableAlreadyExistsException is not thrown, since the table only exists *after* this
+                    // fixed timestamp.
+                }
             } catch (SQLException e) {
                 sqlE = e;
             } finally {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
index 0b6a399..fa01f09 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
@@ -65,11 +65,6 @@ public class DelegateConnectionQueryServices extends DelegateQueryServices imple
     }
 
     @Override
-    public StatsManager getStatsManager() {
-        return getDelegate().getStatsManager();
-    }
-
-    @Override
     public List<HRegionLocation> getAllTableRegions(byte[] tableName) throws SQLException {
         return getDelegate().getAllTableRegions(tableName);
     }
@@ -231,4 +226,15 @@ public class DelegateConnectionQueryServices extends DelegateQueryServices imple
     public String getUserName() {
         return getDelegate().getUserName();
     }
+
+    @Override
+    public long updateStatistics(KeyRange keyRange, byte[] tableName) throws SQLException {
+        return getDelegate().updateStatistics(keyRange, tableName);
+    }
+    
+    @Override
+    public void clearCacheForTable(byte[] tenantId, byte[] schemaName, byte[] tableName, long clientTS)
+            throws SQLException {
+        getDelegate().clearCacheForTable(tenantId, schemaName, tableName, clientTS);
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
index da2d487..bbc653e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
@@ -34,24 +34,31 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DATA_TYPE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DECIMAL_DIGITS;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DEFAULT_COLUMN_FAMILY_NAME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DISABLE_WAL;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.GUIDE_POSTS;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IMMUTABLE_ROWS;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INCREMENT_BY;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_STATE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_TYPE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_AUTOINCREMENT;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_NULLABLE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_VIEW_REFERENCED;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.KEY_SEQ;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.LAST_STATS_UPDATE_TIME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.LIMIT_REACHED_FLAG;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.LINK_TYPE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.MAX_KEY;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.MAX_VALUE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.MIN_KEY;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.MIN_VALUE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.MULTI_TENANT;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.NULLABLE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.NUM_PREC_RADIX;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ORDINAL_POSITION;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.PHYSICAL_NAME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.PK_NAME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.REF_GENERATION;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.REGION_NAME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.REMARKS;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SALT_BUCKETS;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SCOPE_CATALOG;
@@ -67,6 +74,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SQL_DATETIME_SUB;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.START_WITH;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_STATS_TABLE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_NAME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_SCHEM;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_SEQ_NUM;
@@ -78,8 +86,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_CONSTANT;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_INDEX_ID;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_STATEMENT;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_TYPE;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_TYPE;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP;
+
 import java.math.BigDecimal;
 
 import org.apache.hadoop.hbase.HConstants;
@@ -111,6 +118,7 @@ public interface QueryConstants {
     
     public enum JoinType {INNER, LEFT_OUTER}
     public final static String SYSTEM_SCHEMA_NAME = "SYSTEM";
+    public final static byte[] SYSTEM_SCHEMA_NAME_BYTES = Bytes.toBytes(SYSTEM_SCHEMA_NAME);
     public final static String PHOENIX_METADATA = "table";
 
     public final static PName SINGLE_COLUMN_NAME = PNameFactory.newNormalizedName("s");
@@ -220,6 +228,22 @@ public interface QueryConstants {
             HConstants.VERSIONS + "=" + MetaDataProtocol.DEFAULT_MAX_META_DATA_VERSIONS + ",\n" +
             HTableDescriptor.SPLIT_POLICY + "='" + MetaDataSplitPolicy.class.getName() + "'\n";
     
+    public static final String CREATE_STATS_TABLE_METADATA = 
+            "CREATE TABLE " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_STATS_TABLE + "\"(\n" +
+            // PK columns
+            PHYSICAL_NAME  + " VARCHAR NOT NULL," +
+            COLUMN_FAMILY + " VARCHAR," +
+            REGION_NAME + " VARCHAR," +
+            GUIDE_POSTS  + " VARBINARY[]," +
+            MIN_KEY + " VARBINARY," + 
+            MAX_KEY + " VARBINARY," +
+            LAST_STATS_UPDATE_TIME+ " DATE, "+
+            "CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY ("
+            + PHYSICAL_NAME + ","
+            + COLUMN_FAMILY + ","+ REGION_NAME+"))\n" +
+            HConstants.VERSIONS + "=" + MetaDataProtocol.DEFAULT_MAX_META_DATA_VERSIONS + ",\n" +
+            HTableDescriptor.SPLIT_POLICY + "='" + MetaDataSplitPolicy.class.getName() + "'\n";
+            
     public static final String CREATE_SEQUENCE_METADATA =
             "CREATE TABLE " + SYSTEM_CATALOG_SCHEMA + ".\"" + TYPE_SEQUENCE + "\"(\n" +                                    
             TENANT_ID + " VARCHAR NULL," +

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
index 9594f33..fd4152b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
@@ -68,18 +68,14 @@ public interface QueryServices extends SQLCloseable {
     public static final String MAX_MEMORY_WAIT_MS_ATTRIB = "phoenix.query.maxGlobalMemoryWaitMs";
     public static final String MAX_TENANT_MEMORY_PERC_ATTRIB = "phoenix.query.maxTenantMemoryPercentage";
     public static final String MAX_SERVER_CACHE_SIZE_ATTRIB = "phoenix.query.maxServerCacheBytes";
-    public static final String TARGET_QUERY_CONCURRENCY_ATTRIB = "phoenix.query.targetConcurrency";
-    public static final String MAX_QUERY_CONCURRENCY_ATTRIB = "phoenix.query.maxConcurrency";
     public static final String DATE_FORMAT_ATTRIB = "phoenix.query.dateFormat";
     public static final String NUMBER_FORMAT_ATTRIB = "phoenix.query.numberFormat";
     public static final String STATS_UPDATE_FREQ_MS_ATTRIB = "phoenix.query.statsUpdateFrequency";
-    public static final String MAX_STATS_AGE_MS_ATTRIB = "phoenix.query.maxStatsAge";
     public static final String CALL_QUEUE_ROUND_ROBIN_ATTRIB = "ipc.server.callqueue.roundrobin";
     public static final String SCAN_CACHE_SIZE_ATTRIB = "hbase.client.scanner.caching";
     public static final String MAX_MUTATION_SIZE_ATTRIB = "phoenix.mutate.maxSize";
     public static final String MUTATE_BATCH_SIZE_ATTRIB = "phoenix.mutate.batchSize";
     public static final String MAX_SERVER_CACHE_TIME_TO_LIVE_MS_ATTRIB = "phoenix.coprocessor.maxServerCacheTimeToLiveMs";
-    public static final String MAX_INTRA_REGION_PARALLELIZATION_ATTRIB  = "phoenix.query.maxIntraRegionParallelization";
     public static final String ROW_KEY_ORDER_SALTED_TABLE_ATTRIB  = "phoenix.query.rowKeyOrderSaltedTable";
     public static final String USE_INDEXES_ATTRIB  = "phoenix.query.useIndexes";
     public static final String IMMUTABLE_ROWS_ATTRIB  = "phoenix.mutate.immutableRows";
@@ -134,7 +130,9 @@ public interface QueryServices extends SQLCloseable {
     public static final String TRACING_STATS_TABLE_NAME_ATTRIB = "phoenix.trace.statsTableName";
 
     public static final String USE_REVERSE_SCAN_ATTRIB = "phoenix.query.useReverseScan";
-    
+
+    public static final String HISTOGRAM_BYTE_DEPTH_CONF_KEY = "phoenix.guidepost.width";
+
     /**
      * Get executor service used for parallel scans
      */

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
index 605d44c..a0bc4da 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
@@ -24,16 +24,15 @@ import static org.apache.phoenix.query.QueryServices.DROP_METADATA_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.GROUPBY_MAX_CACHE_SIZE_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.GROUPBY_SPILLABLE_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.GROUPBY_SPILL_FILES_ATTRIB;
+import static org.apache.phoenix.query.QueryServices.HISTOGRAM_BYTE_DEPTH_CONF_KEY;
 import static org.apache.phoenix.query.QueryServices.IMMUTABLE_ROWS_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.INDEX_MUTATE_BATCH_SIZE_THRESHOLD_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.KEEP_ALIVE_MS_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.MASTER_INFO_PORT_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.MAX_CLIENT_METADATA_CACHE_SIZE_ATTRIB;
-import static org.apache.phoenix.query.QueryServices.MAX_INTRA_REGION_PARALLELIZATION_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.MAX_MEMORY_PERC_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.MAX_MEMORY_WAIT_MS_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.MAX_MUTATION_SIZE_ATTRIB;
-import static org.apache.phoenix.query.QueryServices.MAX_QUERY_CONCURRENCY_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.MAX_SERVER_CACHE_SIZE_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.MAX_SERVER_CACHE_TIME_TO_LIVE_MS_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.MAX_SERVER_METADATA_CACHE_SIZE_ATTRIB;
@@ -51,7 +50,6 @@ import static org.apache.phoenix.query.QueryServices.SEQUENCE_CACHE_SIZE_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.SPOOL_DIRECTORY;
 import static org.apache.phoenix.query.QueryServices.SPOOL_THRESHOLD_BYTES_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.STATS_UPDATE_FREQ_MS_ATTRIB;
-import static org.apache.phoenix.query.QueryServices.TARGET_QUERY_CONCURRENCY_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.THREAD_POOL_SIZE_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.THREAD_TIMEOUT_MS_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.USE_INDEXES_ATTRIB;
@@ -75,7 +73,7 @@ import org.apache.phoenix.util.ReadOnlyProps;
 public class QueryServicesOptions {
 	public static final int DEFAULT_KEEP_ALIVE_MS = 60000;
 	public static final int DEFAULT_THREAD_POOL_SIZE = 128;
-	public static final int DEFAULT_QUEUE_SIZE = 500;
+	public static final int DEFAULT_QUEUE_SIZE = 5000;
 	public static final int DEFAULT_THREAD_TIMEOUT_MS = 600000; // 10min
 	public static final int DEFAULT_SPOOL_THRESHOLD_BYTES = 1024 * 1024 * 20; // 20m
     public static final String DEFAULT_SPOOL_DIRECTORY = "/tmp";
@@ -146,6 +144,8 @@ public class QueryServicesOptions {
     public static final String DEFAULT_TRACING_STATS_TABLE_NAME = "SYSTEM.TRACING_STATS";
     public static final String DEFAULT_TRACING_FREQ = Tracing.Frequency.NEVER.getKey();
     public static final double DEFAULT_TRACING_PROBABILITY_THRESHOLD = 0.05;
+    public static final long DEFAULT_HISTOGRAM_BYTE_DEPTH = 1024 * 1024;
+    
     
     public static final boolean DEFAULT_USE_REVERSE_SCAN = true;
 
@@ -186,13 +186,10 @@ public class QueryServicesOptions {
             .setIfUnset(MAX_TENANT_MEMORY_PERC_ATTRIB, DEFAULT_MAX_TENANT_MEMORY_PERC)
             .setIfUnset(MAX_SERVER_CACHE_SIZE_ATTRIB, DEFAULT_MAX_SERVER_CACHE_SIZE)
             .setIfUnset(SCAN_CACHE_SIZE_ATTRIB, DEFAULT_SCAN_CACHE_SIZE)
-            .setIfUnset(TARGET_QUERY_CONCURRENCY_ATTRIB, DEFAULT_TARGET_QUERY_CONCURRENCY)
-            .setIfUnset(MAX_QUERY_CONCURRENCY_ATTRIB, DEFAULT_MAX_QUERY_CONCURRENCY)
             .setIfUnset(DATE_FORMAT_ATTRIB, DEFAULT_DATE_FORMAT)
             .setIfUnset(STATS_UPDATE_FREQ_MS_ATTRIB, DEFAULT_STATS_UPDATE_FREQ_MS)
             .setIfUnset(CALL_QUEUE_ROUND_ROBIN_ATTRIB, DEFAULT_CALL_QUEUE_ROUND_ROBIN)
             .setIfUnset(MAX_MUTATION_SIZE_ATTRIB, DEFAULT_MAX_MUTATION_SIZE)
-            .setIfUnset(MAX_INTRA_REGION_PARALLELIZATION_ATTRIB, DEFAULT_MAX_INTRA_REGION_PARALLELIZATION)
             .setIfUnset(ROW_KEY_ORDER_SALTED_TABLE_ATTRIB, DEFAULT_ROW_KEY_ORDER_SALTED_TABLE)
             .setIfUnset(USE_INDEXES_ATTRIB, DEFAULT_USE_INDEXES)
             .setIfUnset(IMMUTABLE_ROWS_ATTRIB, DEFAULT_IMMUTABLE_ROWS)
@@ -204,6 +201,7 @@ public class QueryServicesOptions {
             .setIfUnset(GROUPBY_SPILL_FILES_ATTRIB, DEFAULT_GROUPBY_SPILL_FILES)
             .setIfUnset(SEQUENCE_CACHE_SIZE_ATTRIB, DEFAULT_SEQUENCE_CACHE_SIZE)
             .setIfUnset(SCAN_RESULT_CHUNK_SIZE, DEFAULT_SCAN_RESULT_CHUNK_SIZE)
+            .setIfUnset(HISTOGRAM_BYTE_DEPTH_CONF_KEY, DEFAULT_HISTOGRAM_BYTE_DEPTH);
             ;
         // HBase sets this to 1, so we reset it to something more appropriate.
         // Hopefully HBase will change this, because we can't know if a user set
@@ -294,14 +292,6 @@ public class QueryServicesOptions {
         return set(SCAN_CACHE_SIZE_ATTRIB, scanFetchSize);
     }
     
-    public QueryServicesOptions setMaxQueryConcurrency(int maxQueryConcurrency) {
-        return set(MAX_QUERY_CONCURRENCY_ATTRIB, maxQueryConcurrency);
-    }
-
-    public QueryServicesOptions setTargetQueryConcurrency(int targetQueryConcurrency) {
-        return set(TARGET_QUERY_CONCURRENCY_ATTRIB, targetQueryConcurrency);
-    }
-    
     public QueryServicesOptions setDateFormat(String dateFormat) {
         return set(DATE_FORMAT_ATTRIB, dateFormat);
     }
@@ -310,6 +300,10 @@ public class QueryServicesOptions {
         return set(STATS_UPDATE_FREQ_MS_ATTRIB, frequencyMs);
     }
     
+    public QueryServicesOptions setHistogramDepthBytes(int depth) {
+        return set(HISTOGRAM_BYTE_DEPTH_CONF_KEY, depth);
+    }
+    
     public QueryServicesOptions setCallQueueRoundRobin(boolean isRoundRobin) {
         return set(CALL_QUEUE_PRODUCER_ATTRIB_NAME, isRoundRobin);
     }
@@ -322,10 +316,6 @@ public class QueryServicesOptions {
         return set(MUTATE_BATCH_SIZE_ATTRIB, mutateBatchSize);
     }
     
-    public QueryServicesOptions setMaxIntraRegionParallelization(int maxIntraRegionParallelization) {
-        return set(MAX_INTRA_REGION_PARALLELIZATION_ATTRIB, maxIntraRegionParallelization);
-    }
-    
     public QueryServicesOptions setRowKeyOrderSaltedTable(boolean rowKeyOrderSaltedTable) {
         return set(ROW_KEY_ORDER_SALTED_TABLE_ATTRIB, rowKeyOrderSaltedTable);
     }
@@ -394,11 +384,7 @@ public class QueryServicesOptions {
     public int getMutateBatchSize() {
         return config.getInt(MUTATE_BATCH_SIZE_ATTRIB, DEFAULT_MUTATE_BATCH_SIZE);
     }
-    
-    public int getMaxIntraRegionParallelization() {
-        return config.getInt(MAX_INTRA_REGION_PARALLELIZATION_ATTRIB, DEFAULT_MAX_INTRA_REGION_PARALLELIZATION);
-    }
-    
+
     public boolean isUseIndexes() {
         return config.getBoolean(USE_INDEXES_ATTRIB, DEFAULT_USE_INDEXES);
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/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 56b6604..eafac8b 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
@@ -37,15 +37,19 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_STATE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_TYPE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_VIEW_REFERENCED;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.KEY_SEQ;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.LAST_STATS_UPDATE_TIME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.LINK_TYPE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.MULTI_TENANT;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.NULLABLE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ORDINAL_POSITION;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.PHYSICAL_NAME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.PK_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.REGION_NAME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SALT_BUCKETS;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SORT_ORDER;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_STATS_TABLE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_NAME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_SCHEM;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_SEQ_NUM;
@@ -60,8 +64,10 @@ import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_DROP_METADAT
 import static org.apache.phoenix.schema.PDataType.VARCHAR;
 
 import java.io.IOException;
+import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.PreparedStatement;
+import java.sql.ResultSet;
 import java.sql.ResultSetMetaData;
 import java.sql.SQLException;
 import java.sql.SQLFeatureNotSupportedException;
@@ -123,6 +129,8 @@ import org.apache.phoenix.parse.NamedTableNode;
 import org.apache.phoenix.parse.ParseNodeFactory;
 import org.apache.phoenix.parse.PrimaryKeyConstraint;
 import org.apache.phoenix.parse.TableName;
+import org.apache.phoenix.parse.UpdateStatisticsStatement;
+import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
@@ -133,6 +141,7 @@ import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.ScanUtil;
 import org.apache.phoenix.util.SchemaUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -462,6 +471,55 @@ public class MetaDataClient {
         return connection.getQueryServices().updateData(plan);
     }
 
+    public MutationState updateStatistics(UpdateStatisticsStatement updateStatisticsStmt) throws SQLException {
+        // Check before updating the stats if we have reached the configured time to reupdate the stats once again
+        long minTimeForStatsUpdate = connection.getQueryServices().getProps()
+                .getLong(QueryServices.STATS_UPDATE_FREQ_MS_ATTRIB, QueryServicesOptions.DEFAULT_STATS_UPDATE_FREQ_MS);
+        ColumnResolver resolver = FromCompiler.getResolver(updateStatisticsStmt, connection);
+        PTable table = resolver.getTables().get(0).getTable();
+        PName physicalName = table.getPhysicalName();
+        byte[] tenantIdBytes = ByteUtil.EMPTY_BYTE_ARRAY;
+        KeyRange analyzeRange = KeyRange.EVERYTHING_RANGE;
+        if (connection.getTenantId() != null && table.isMultiTenant()) {
+            tenantIdBytes = connection.getTenantId().getBytes();
+            //  TODO remove this inner if once PHOENIX-1259 is fixed.
+            if (table.getBucketNum() == null && table.getIndexType() != IndexType.LOCAL) {
+                List<List<KeyRange>> tenantIdKeyRanges = Collections.singletonList(Collections.singletonList(KeyRange
+                        .getKeyRange(tenantIdBytes)));
+                byte[] lowerRange = ScanUtil.getMinKey(table.getRowKeySchema(), tenantIdKeyRanges,
+                        ScanUtil.SINGLE_COLUMN_SLOT_SPAN);
+                byte[] upperRange = ScanUtil.getMaxKey(table.getRowKeySchema(), tenantIdKeyRanges,
+                        ScanUtil.SINGLE_COLUMN_SLOT_SPAN);
+                analyzeRange = KeyRange.getKeyRange(lowerRange, upperRange);
+            }
+        }
+        Long scn = connection.getSCN();
+        // Always invalidate the cache
+        long clientTS = connection.getSCN() == null ? HConstants.LATEST_TIMESTAMP : scn;
+        connection.getQueryServices().clearCacheForTable(tenantIdBytes, table.getSchemaName().getBytes(),
+                table.getTableName().getBytes(), clientTS);
+        // Clear the cache also. So that for cases like major compaction also we would be able to use the stats
+        updateCache(table.getSchemaName().getString(), table.getTableName().getString(), true);
+        String query = "SELECT CURRENT_DATE(),"+ LAST_STATS_UPDATE_TIME + " FROM " + SYSTEM_CATALOG_SCHEMA
+                + "." + SYSTEM_STATS_TABLE + " WHERE " + PHYSICAL_NAME + "='" + physicalName.getString() + "' AND " + COLUMN_FAMILY
+                + " IS NULL AND " + REGION_NAME + " IS NULL";
+        ResultSet rs = connection.createStatement().executeQuery(query);
+        long lastUpdatedTime = 0;
+        if (rs.next() && rs.getDate(2) != null) {
+            lastUpdatedTime = rs.getDate(1).getTime() - rs.getDate(2).getTime();
+        }
+        if (minTimeForStatsUpdate  > lastUpdatedTime) {
+            // We need to update the stats table
+            connection.getQueryServices().updateStatistics(analyzeRange, physicalName.getBytes());
+            connection.getQueryServices().clearCacheForTable(tenantIdBytes, table.getSchemaName().getBytes(),
+                    table.getTableName().getBytes(), clientTS);
+            updateCache(table.getSchemaName().getString(), table.getTableName().getString(), true);
+            return new MutationState(1, connection);
+        } else {
+            return new MutationState(0, connection);
+        }
+    }
+
     private MutationState buildIndexAtTimeStamp(PTable index, NamedTableNode dataTableNode) throws SQLException {
         // If our connection is at a fixed point-in-time, we need to open a new
         // connection so that our new index table is visible.
@@ -1440,7 +1498,8 @@ public class MetaDataClient {
         return dropTable(schemaName, tableName, parentTableName, PTableType.INDEX, statement.ifExists(), false);
     }
 
-    private MutationState dropTable(String schemaName, String tableName, String parentTableName, PTableType tableType, boolean ifExists, boolean cascade) throws SQLException {
+    private MutationState dropTable(String schemaName, String tableName, String parentTableName, PTableType tableType,
+            boolean ifExists, boolean cascade) throws SQLException {
         connection.rollback();
         boolean wasAutoCommit = connection.getAutoCommit();
         try {
@@ -1465,80 +1524,118 @@ public class MetaDataClient {
 
             MetaDataMutationResult result = connection.getQueryServices().dropTable(tableMetaData, tableType, cascade);
             MutationCode code = result.getMutationCode();
-            switch(code) {
-                case TABLE_NOT_FOUND:
-                    if (!ifExists) {
-                        throw new TableNotFoundException(schemaName, tableName);
-                    }
-                    break;
-                case NEWER_TABLE_FOUND:
-                    throw new NewerTableAlreadyExistsException(schemaName, tableName);
-                case UNALLOWED_TABLE_MUTATION:
-                    throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_MUTATE_TABLE)
-                        .setSchemaName(schemaName).setTableName(tableName).build().buildException();
-                default:
-                    connection.removeTable(tenantId, SchemaUtil.getTableName(schemaName, tableName), parentTableName, result.getMutationTime());
-                                        
-                    if (result.getTable() != null && tableType != PTableType.VIEW) {
-                        connection.setAutoCommit(true);
-                        PTable table = result.getTable();
-                        boolean dropMetaData = result.getTable().getViewIndexId() == null && 
-                                connection.getQueryServices().getProps().getBoolean(DROP_METADATA_ATTRIB, DEFAULT_DROP_METADATA);
-                        long ts = (scn == null ? result.getMutationTime() : scn);
-                        // Create empty table and schema - they're only used to get the name from
-                        // PName name, PTableType type, long timeStamp, long sequenceNumber, List<PColumn> columns
-                        List<TableRef> tableRefs = Lists.newArrayListWithExpectedSize(2 + table.getIndexes().size());
-                        // All multi-tenant tables have a view index table, so no need to check in that case
-                        if (tableType == PTableType.TABLE && (table.isMultiTenant() || hasViewIndexTable || hasLocalIndexTable)) {
-                            MetaDataUtil.deleteViewIndexSequences(connection, table.getPhysicalName());
-                            // TODO: consider removing this, as the DROP INDEX done for each DROP VIEW command
-                            // would have deleted all the rows already
-                            if (!dropMetaData) {
-                                if (hasViewIndexTable) {
-                                    String viewIndexSchemaName = null;
-                                    String viewIndexTableName = null;
-                                    if(schemaName != null) {
-                                        viewIndexSchemaName = MetaDataUtil.getViewIndexTableName(schemaName);
-                                        viewIndexTableName = tableName;
-                                    } else {
-                                        viewIndexTableName = MetaDataUtil.getViewIndexTableName(tableName);
-                                    }
-                                    PTable viewIndexTable = new PTableImpl(null, viewIndexSchemaName, viewIndexTableName, ts, table.getColumnFamilies());
-                                    tableRefs.add(new TableRef(null, viewIndexTable, ts, false));
-                                } 
-                                if (hasLocalIndexTable) {
-                                    String localIndexSchemaName = null;
-                                    String localIndexTableName = null;
-                                    if(schemaName != null) {
-                                        localIndexSchemaName = MetaDataUtil.getLocalIndexTableName(schemaName);
-                                        localIndexTableName = tableName;
-                                    } else {
-                                        localIndexTableName = MetaDataUtil.getLocalIndexTableName(tableName);
-                                    }
-                                    PTable localIndexTable = new PTableImpl(null, localIndexSchemaName, localIndexTableName, ts, Collections.<PColumnFamily>emptyList());
-                                    tableRefs.add(new TableRef(null, localIndexTable, ts, false));
-                                }
+            switch (code) {
+            case TABLE_NOT_FOUND:
+                if (!ifExists) { throw new TableNotFoundException(schemaName, tableName); }
+                break;
+            case NEWER_TABLE_FOUND:
+                throw new NewerTableAlreadyExistsException(schemaName, tableName);
+            case UNALLOWED_TABLE_MUTATION:
+                throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_MUTATE_TABLE)
+
+                .setSchemaName(schemaName).setTableName(tableName).build().buildException();
+            default:
+                connection.removeTable(tenantId, SchemaUtil.getTableName(schemaName, tableName), parentTableName,
+                        result.getMutationTime());
+
+                if (result.getTable() != null && tableType != PTableType.VIEW) {
+                    connection.setAutoCommit(true);
+                    PTable table = result.getTable();
+                    boolean dropMetaData = result.getTable().getViewIndexId() == null &&
+
+                    connection.getQueryServices().getProps().getBoolean(DROP_METADATA_ATTRIB, DEFAULT_DROP_METADATA);
+                    long ts = (scn == null ? result.getMutationTime() : scn);
+                    // Create empty table and schema - they're only used to get the name from
+                    // PName name, PTableType type, long timeStamp, long sequenceNumber, List<PColumn> columns
+                    List<TableRef> tableRefs = Lists.newArrayListWithExpectedSize(2 + table.getIndexes().size());
+                    // All multi-tenant tables have a view index table, so no need to check in that case
+                    if (tableType == PTableType.TABLE
+                            && (table.isMultiTenant() || hasViewIndexTable || hasLocalIndexTable)) {
+
+                        MetaDataUtil.deleteViewIndexSequences(connection, table.getPhysicalName());
+                        if (hasViewIndexTable) {
+                            String viewIndexSchemaName = null;
+                            String viewIndexTableName = null;
+                            if (schemaName != null) {
+                                viewIndexSchemaName = MetaDataUtil.getViewIndexTableName(schemaName);
+                                viewIndexTableName = tableName;
+                            } else {
+                                viewIndexTableName = MetaDataUtil.getViewIndexTableName(tableName);
                             }
+                            PTable viewIndexTable = new PTableImpl(null, viewIndexSchemaName, viewIndexTableName, ts,
+                                    table.getColumnFamilies());
+                            tableRefs.add(new TableRef(null, viewIndexTable, ts, false));
                         }
-                        if (!dropMetaData) {
-                            // Delete everything in the column. You'll still be able to do queries at earlier timestamps
-                            tableRefs.add(new TableRef(null, table, ts, false));
-                            // TODO: Let the standard mutable secondary index maintenance handle this?
-                            for (PTable index: table.getIndexes()) {
-                                tableRefs.add(new TableRef(null, index, ts, false));
+                        if (hasLocalIndexTable) {
+                            String localIndexSchemaName = null;
+                            String localIndexTableName = null;
+                            if (schemaName != null) {
+                                localIndexSchemaName = MetaDataUtil.getLocalIndexTableName(schemaName);
+                                localIndexTableName = tableName;
+                            } else {
+                                localIndexTableName = MetaDataUtil.getLocalIndexTableName(tableName);
                             }
-                            MutationPlan plan = new PostDDLCompiler(connection).compile(tableRefs, null, null, Collections.<PColumn>emptyList(), ts);
-                            return connection.getQueryServices().updateData(plan);
+                            PTable localIndexTable = new PTableImpl(null, localIndexSchemaName, localIndexTableName,
+                                    ts, Collections.<PColumnFamily> emptyList());
+                            tableRefs.add(new TableRef(null, localIndexTable, ts, false));
                         }
                     }
-                    break;
+                    tableRefs.add(new TableRef(null, table, ts, false));
+                    // TODO: Let the standard mutable secondary index maintenance handle this?
+                    for (PTable index : table.getIndexes()) {
+                        tableRefs.add(new TableRef(null, index, ts, false));
+                    }
+                    deleteFromStatsTable(tableRefs, ts);
+                    if (!dropMetaData) {
+                        MutationPlan plan = new PostDDLCompiler(connection).compile(tableRefs, null, null,
+                                Collections.<PColumn> emptyList(), ts);
+                        // Delete everything in the column. You'll still be able to do queries at earlier timestamps
+                        return connection.getQueryServices().updateData(plan);
+                    }
                 }
-                 return new MutationState(0,connection);
+                break;
+            }
+            return new MutationState(0, connection);
         } finally {
             connection.setAutoCommit(wasAutoCommit);
         }
     }
 
+    private void deleteFromStatsTable(List<TableRef> tableRefs, long ts) throws SQLException {
+        Properties props = new Properties(connection.getClientInfo());
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts));
+        Connection conn = DriverManager.getConnection(connection.getURL(), props);
+        conn.setAutoCommit(true);
+        boolean success = false;
+        SQLException sqlException = null;
+        try {
+            StringBuilder buf = new StringBuilder("DELETE FROM SYSTEM.STATS WHERE PHYSICAL_NAME IN (");
+            for (TableRef ref : tableRefs) {
+                buf.append("'" + ref.getTable().getName().getString() + "',");
+            }
+            buf.setCharAt(buf.length() - 1, ')');
+            conn.createStatement().execute(buf.toString());
+            success = true;
+        } catch (SQLException e) {
+            sqlException = e;
+        } finally {
+            try {
+                conn.close();
+            } catch (SQLException e) {
+                if (sqlException == null) {
+                    // If we're not in the middle of throwing another exception
+                    // then throw the exception we got on close.
+                    if (success) {
+                        sqlException = e;
+                    }
+                } else {
+                    sqlException.setNextException(e);
+                }
+            }
+            if (sqlException != null) { throw sqlException; }
+        }
+    }
+
     private MutationCode processMutationResult(String schemaName, String tableName, MetaDataMutationResult result) throws SQLException {
         final MutationCode mutationCode = result.getMutationCode();
         PName tenantId = connection.getTenantId();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnFamily.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnFamily.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnFamily.java
index 24da14d..01c236f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnFamily.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnFamily.java
@@ -18,6 +18,7 @@
 package org.apache.phoenix.schema;
 
 import java.util.Collection;
+import java.util.List;
 
 /**
  * 
@@ -51,4 +52,6 @@ public interface PColumnFamily {
     PColumn getColumn(String name) throws ColumnNotFoundException;
     
     int getEstimatedSize();
+    
+    List<byte[]> getGuidePosts();
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnFamilyImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnFamilyImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnFamilyImpl.java
index 5ccd50b..15ac8fa 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnFamilyImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnFamilyImpl.java
@@ -17,6 +17,7 @@
  */
 package org.apache.phoenix.schema;
 
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 
@@ -34,6 +35,7 @@ public class PColumnFamilyImpl implements PColumnFamily {
     private final Map<String, PColumn> columnByString;
     private final Map<byte[], PColumn> columnByBytes;
     private final int estimatedSize;
+    private List<byte[]> guidePosts = Collections.emptyList();
 
     @Override
     public int getEstimatedSize() {
@@ -41,9 +43,23 @@ public class PColumnFamilyImpl implements PColumnFamily {
     }
     
     public PColumnFamilyImpl(PName name, List<PColumn> columns) {
+       this(name, columns, null);
+    }
+    
+    public PColumnFamilyImpl(PName name, List<PColumn> columns, List<byte[]> guidePosts) {
         Preconditions.checkNotNull(name);
-        long estimatedSize = SizedUtil.OBJECT_SIZE + SizedUtil.POINTER_SIZE * 4 + SizedUtil.INT_SIZE + name.getEstimatedSize() +
-                SizedUtil.sizeOfMap(columns.size()) * 2 + SizedUtil.sizeOfArrayList(columns.size());
+        // Include guidePosts also in estimating the size
+        int guidePostsSize = 0;
+        if(guidePosts != null) {
+            guidePostsSize = guidePosts.size();
+            for(byte[] gps : guidePosts) {
+                guidePostsSize += gps.length;
+            }
+            Collections.sort(guidePosts, Bytes.BYTES_COMPARATOR);
+            this.guidePosts = guidePosts;
+        }
+        long estimatedSize = SizedUtil.OBJECT_SIZE + SizedUtil.POINTER_SIZE * 5 + SizedUtil.INT_SIZE + name.getEstimatedSize() +
+                SizedUtil.sizeOfMap(columns.size()) * 2 + SizedUtil.sizeOfArrayList(columns.size()) + SizedUtil.sizeOfArrayList(guidePostsSize);
         this.name = name;
         this.columns = ImmutableList.copyOf(columns);
         ImmutableMap.Builder<String, PColumn> columnByStringBuilder = ImmutableMap.builder();
@@ -85,4 +101,9 @@ public class PColumnFamilyImpl implements PColumnFamily {
         }
         return column;
     }
+
+    @Override
+    public List<byte[]> getGuidePosts() {
+        return guidePosts;
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/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 cbf0dad..374b10c 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
@@ -23,7 +23,6 @@ import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.hbase.index.util.KeyValueBuilder;
 import org.apache.phoenix.index.IndexMaintainer;
-import org.apache.phoenix.schema.stat.PTableStats;
 
 
 /**
@@ -253,10 +252,11 @@ public interface PTable {
     int newKey(ImmutableBytesWritable key, byte[][] values);
 
     /**
-     * Return the statistics table associated with this PTable.
+     * Return the statistics table associated with this PTable. A list of 
+     * guide posts are return 
      * @return the statistics table.
      */
-    PTableStats getTableStats();
+    List<byte[]> getGuidePosts();
 
     RowKeySchema getRowKeySchema();
 
@@ -316,4 +316,5 @@ public interface PTable {
     
     int getEstimatedSize();
     IndexType getIndexType();
+    
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/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 abe637d..41faaf2 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
@@ -26,11 +26,10 @@ import java.io.IOException;
 import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-import java.util.Map.Entry;
+import java.util.TreeMap;
 
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HConstants;
@@ -96,8 +95,6 @@ public class PTableImpl implements PTable {
     private ListMultimap<String,PColumn> columnsByName;
     private PName pkName;
     private Integer bucketNum;
-    // Statistics associated with this table.
-    private PTableStats stats;
     private RowKeySchema rowKeySchema;
     // Indexes associated with this table.
     private List<PTable> indexes;
@@ -116,6 +113,7 @@ public class PTableImpl implements PTable {
     private Short viewIndexId;
     private int estimatedSize;
     private IndexType indexType;
+    private List<byte[]> guidePosts = Collections.emptyList();
     
     public PTableImpl() {
         this.indexes = Collections.emptyList();
@@ -213,6 +211,17 @@ public class PTableImpl implements PTable {
         return new PTableImpl(tenantId, schemaName, tableName, type, state, timeStamp, sequenceNumber, pkName, bucketNum, columns, dataTableName,
                 indexes, isImmutableRows, physicalNames, defaultFamilyName, viewExpression, disableWAL, multiTenant, viewType, viewIndexId, indexType);
     }
+    
+    public static PTableImpl makePTable(PName tenantId, PName schemaName, PName tableName, PTableType type,
+            PIndexState state, long timeStamp, long sequenceNumber, PName pkName, Integer bucketNum,
+            List<PColumn> columns, PName dataTableName, List<PTable> indexes, boolean isImmutableRows,
+            List<PName> physicalNames, PName defaultFamilyName, String viewExpression, boolean disableWAL,
+            boolean multiTenant, ViewType viewType, Short viewIndexId, IndexType indexType, PTableStats stats)
+            throws SQLException {
+        return new PTableImpl(tenantId, schemaName, tableName, type, state, timeStamp, sequenceNumber, pkName,
+                bucketNum, columns, dataTableName, indexes, isImmutableRows, physicalNames, defaultFamilyName,
+                viewExpression, disableWAL, multiTenant, viewType, viewIndexId, indexType, stats);
+    }
 
     private PTableImpl(PName tenantId, PName schemaName, PName tableName, PTableType type, PIndexState state, long timeStamp, long sequenceNumber,
             PName pkName, Integer bucketNum, List<PColumn> columns, PName dataTableName, List<PTable> indexes, boolean isImmutableRows,
@@ -220,6 +229,16 @@ public class PTableImpl implements PTable {
         init(tenantId, schemaName, tableName, type, state, timeStamp, sequenceNumber, pkName, bucketNum, columns,
                 new PTableStatsImpl(), dataTableName, indexes, isImmutableRows, physicalNames, defaultFamilyName, viewExpression, disableWAL, multiTenant, viewType, viewIndexId, indexType);
     }
+    
+    private PTableImpl(PName tenantId, PName schemaName, PName tableName, PTableType type, PIndexState state,
+            long timeStamp, long sequenceNumber, PName pkName, Integer bucketNum, List<PColumn> columns,
+            PName dataTableName, List<PTable> indexes, boolean isImmutableRows, List<PName> physicalNames,
+            PName defaultFamilyName, String viewExpression, boolean disableWAL, boolean multiTenant, ViewType viewType,
+            Short viewIndexId, IndexType indexType, PTableStats stats) throws SQLException {
+        init(tenantId, schemaName, tableName, type, state, timeStamp, sequenceNumber, pkName, bucketNum, columns,
+                stats, dataTableName, indexes, isImmutableRows, physicalNames, defaultFamilyName, viewExpression,
+                disableWAL, multiTenant, viewType, viewIndexId, indexType);
+    }
 
     @Override
     public boolean isMultiTenant() {
@@ -240,7 +259,7 @@ public class PTableImpl implements PTable {
     private void init(PName tenantId, PName schemaName, PName tableName, PTableType type, PIndexState state, long timeStamp, long sequenceNumber,
             PName pkName, Integer bucketNum, List<PColumn> columns, PTableStats stats, PName parentTableName, List<PTable> indexes,
             boolean isImmutableRows, List<PName> physicalNames, PName defaultFamilyName, String viewExpression, boolean disableWAL, boolean multiTenant,
-            ViewType viewType, Short viewIndexId, IndexType indexType) throws SQLException {
+            ViewType viewType, Short viewIndexId, IndexType indexType ) 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 +
@@ -330,16 +349,40 @@ public class PTableImpl implements PTable {
                 columnsInFamily.add(column);
             }
         }
-        
         this.rowKeySchema = builder.build();
         estimatedSize += rowKeySchema.getEstimatedSize();
         Iterator<Map.Entry<PName,List<PColumn>>> iterator = familyMap.entrySet().iterator();
         PColumnFamily[] families = new PColumnFamily[familyMap.size()];
+        if (families.length == 0) {
+            if(stats != null) {
+                byte[] defaultFamilyNameBytes = null;
+                if(defaultFamilyName == null) {
+                    defaultFamilyNameBytes = QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES;
+                } else {
+                    defaultFamilyNameBytes = defaultFamilyName.getBytes();
+                }
+                if (stats.getGuidePosts().get(defaultFamilyNameBytes) != null) {
+                    guidePosts = stats.getGuidePosts().get(defaultFamilyNameBytes);
+                    if (guidePosts != null) {
+                        Collections.sort(guidePosts, Bytes.BYTES_COMPARATOR);
+                        estimatedSize += SizedUtil.sizeOfArrayList(guidePosts.size());
+                        for (byte[] gps : guidePosts) {
+                            estimatedSize += gps.length;
+                        }
+                    }
+                }
+            }
+        }
         ImmutableMap.Builder<String, PColumnFamily> familyByString = ImmutableMap.builder();
-        ImmutableSortedMap.Builder<byte[], PColumnFamily> familyByBytes = ImmutableSortedMap.orderedBy(Bytes.BYTES_COMPARATOR);
+        ImmutableSortedMap.Builder<byte[], PColumnFamily> familyByBytes = ImmutableSortedMap
+                .orderedBy(Bytes.BYTES_COMPARATOR);
+        List<byte[]> famGuidePosts = null;
         for (int i = 0; i < families.length; i++) {
             Map.Entry<PName,List<PColumn>> entry = iterator.next();
-            PColumnFamily family = new PColumnFamilyImpl(entry.getKey(), entry.getValue());
+            if (stats != null) {
+                famGuidePosts = stats.getGuidePosts().get(entry.getKey().getBytes());
+            }
+            PColumnFamily family = new PColumnFamilyImpl(entry.getKey(), entry.getValue(), famGuidePosts);
             families[i] = family;
             familyByString.put(family.getName().getString(), family);
             familyByBytes.put(family.getName().getBytes(), family);
@@ -350,8 +393,6 @@ public class PTableImpl implements PTable {
         this.familyByString = familyByString.build();
         estimatedSize += SizedUtil.sizeOfArrayList(families.length);
         estimatedSize += SizedUtil.sizeOfMap(families.length) * 2;
-        
-        this.stats = stats;
         this.indexes = indexes == null ? Collections.<PTable>emptyList() : indexes;
         for (PTable index : this.indexes) {
             estimatedSize += index.getEstimatedSize();
@@ -693,8 +734,8 @@ public class PTableImpl implements PTable {
     }
 
     @Override
-    public PTableStats getTableStats() {
-        return stats;
+    public List<byte[]> getGuidePosts() {
+        return guidePosts;
     }
 
     @Override
@@ -851,14 +892,15 @@ public class PTableImpl implements PTable {
       for (PTableProtos.PTable curPTableProto : table.getIndexesList()) {
         indexes.add(createFromProto(curPTableProto));
       }
+      
       boolean isImmutableRows = table.getIsImmutableRows();
-      Map<String, byte[][]> guidePosts = new HashMap<String, byte[][]>();
+      TreeMap<byte[], List<byte[]>> tableGuidePosts = new TreeMap<byte[], List<byte[]>>(Bytes.BYTES_COMPARATOR);
       for (PTableProtos.PTableStats pTableStatsProto : table.getGuidePostsList()) {
-        byte[][] value = new byte[pTableStatsProto.getValuesCount()][];
-        for (int j = 0; j < pTableStatsProto.getValuesCount(); j++) {
-          value[j] = pTableStatsProto.getValues(j).toByteArray();
-        }
-        guidePosts.put(pTableStatsProto.getKey(), value);
+          List<byte[]> value = Lists.newArrayListWithExpectedSize(pTableStatsProto.getValuesCount());
+            for (int j = 0; j < pTableStatsProto.getValuesCount(); j++) {
+                value.add(pTableStatsProto.getValues(j).toByteArray());
+            }
+            tableGuidePosts.put(pTableStatsProto.getKeyBytes().toByteArray(), value);
       }
       PName dataTableName = null;
       if (table.hasDataTableNameBytes()) {
@@ -886,7 +928,7 @@ public class PTableImpl implements PTable {
         }
       }
       
-      PTableStats stats = new PTableStatsImpl(guidePosts);
+      PTableStats stats = new PTableStatsImpl(tableGuidePosts);
       try {
         PTableImpl result = new PTableImpl();
         result.init(tenantId, schemaName, tableName, tableType, indexState, timeStamp, sequenceNumber, pkName,
@@ -937,24 +979,37 @@ public class PTableImpl implements PTable {
         PColumn column = columns.get(i);
         builder.addColumns(PColumnImpl.toProto(column));
       }
+
       List<PTable> indexes = table.getIndexes();
       for (PTable curIndex : indexes) {
         builder.addIndexes(toProto(curIndex));
       }
       builder.setIsImmutableRows(table.isImmutableRows());
 
-      // build stats
-      Map<String, byte[][]> statsMap = table.getTableStats().getGuidePosts();
-      if(statsMap != null) {
-        for (Entry<String, byte[][]> entry : statsMap.entrySet()) {
-          PTableProtos.PTableStats.Builder statsBuilder = PTableProtos.PTableStats.newBuilder();
-          statsBuilder.setKey(entry.getKey());
-          for (byte[] curVal : entry.getValue()) {
-            statsBuilder.addValues(HBaseZeroCopyByteString.wrap(curVal));
-          }
-          builder.addGuidePosts(statsBuilder.build());
-        }
-      }
+        // build stats for the table
+      if (table.getColumnFamilies().isEmpty() && !table.getGuidePosts().isEmpty()) {
+         List<byte[]> stats = table.getGuidePosts();
+          if (stats != null) {
+             PTableProtos.PTableStats.Builder statsBuilder = PTableProtos.PTableStats.newBuilder();
+             statsBuilder.setKey(Bytes.toString(SchemaUtil.getEmptyColumnFamily(table)));
+             for (byte[] stat : stats) {
+                 statsBuilder.addValues(HBaseZeroCopyByteString.wrap(stat));
+             }
+             builder.addGuidePosts(statsBuilder.build());
+         }
+      } else {
+            for (PColumnFamily fam : table.getColumnFamilies()) {
+                PTableProtos.PTableStats.Builder statsBuilder = PTableProtos.PTableStats.newBuilder();
+                if (fam.getGuidePosts() != null) {
+                    statsBuilder.setKey(fam.getName().getString());
+                    for (byte[] stat : fam.getGuidePosts()) {
+                        statsBuilder.addValues(HBaseZeroCopyByteString.wrap(stat));
+                    }
+                    builder.addGuidePosts(statsBuilder.build());
+                }
+            }
+       }
+
       if (table.getParentName() != null) {
         builder.setDataTableNameBytes(HBaseZeroCopyByteString.wrap(table.getParentTableName().getBytes()));
       }
@@ -980,4 +1035,5 @@ public class PTableImpl implements PTable {
     public PTableKey getKey() {
         return key;
     }
+    
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/main/java/org/apache/phoenix/schema/stat/PTableStats.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/stat/PTableStats.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/stat/PTableStats.java
index 5d867ef..dfda457 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/stat/PTableStats.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/stat/PTableStats.java
@@ -17,28 +17,19 @@
  */
 package org.apache.phoenix.schema.stat;
 
-import java.util.Map;
+import java.util.List;
+import java.util.TreeMap;
 
-import org.apache.hadoop.hbase.HRegionInfo;
 
-
-/**
- * Interface for Phoenix table statistics. Statistics is collected on the server
- * side and can be used for various purpose like splitting region for scanning, etc.
- * 
- * The table is defined on the client side, but it is populated on the server side. The client
- * should not populate any data to the statistics object.
+/*
+ * The table is defined on the client side, but it is populated on the server side. The client should not populate any data to the
+ * statistics object.
  */
 public interface PTableStats {
-
     /**
-     * Given the region info, returns an array of bytes that is the current estimate of key
-     * distribution inside that region. The keys should split that region into equal chunks.
-     * 
-     * @param region
-     * @return array of keys
+     * Returns a tree map of the guide posts collected against a column family
+     * @return
      */
-    byte[][] getRegionGuidePosts(HRegionInfo region);
+    TreeMap<byte[], List<byte[]>> getGuidePosts();
 
-    Map<String, byte[][]> getGuidePosts();
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/main/java/org/apache/phoenix/schema/stat/PTableStatsImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/stat/PTableStatsImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/stat/PTableStatsImpl.java
index 8690a17..3e8f1e9 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/stat/PTableStatsImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/stat/PTableStatsImpl.java
@@ -16,40 +16,31 @@
  * limitations under the License.
  */
 package org.apache.phoenix.schema.stat;
+ import java.util.List;
+import java.util.TreeMap;
 
-import java.util.Map;
-
-import org.apache.hadoop.hbase.HRegionInfo;
-
-import com.google.common.collect.ImmutableMap;
-
-
-/**
+import org.apache.hadoop.hbase.util.Bytes;
+ 
+ /**
  * Implementation for PTableStats.
  */
 public class PTableStatsImpl implements PTableStats {
 
-    // The map for guide posts should be immutable. We only take the current snapshot from outside
-    // method call and store it.
-    private Map<String, byte[][]> regionGuidePosts;
+    public static final PTableStats NO_STATS = new PTableStatsImpl();
 
-    public PTableStatsImpl() { }
+    private TreeMap<byte[], List<byte[]>> guidePosts = new TreeMap<byte[], List<byte[]>>(Bytes.BYTES_COMPARATOR);
 
-    public PTableStatsImpl(Map<String, byte[][]> stats) {
-        regionGuidePosts = ImmutableMap.copyOf(stats);
+    public PTableStatsImpl() {
+        this(new TreeMap<byte[], List<byte[]>>(Bytes.BYTES_COMPARATOR));
     }
 
-    @Override
-    public byte[][] getRegionGuidePosts(HRegionInfo region) {
-        return regionGuidePosts.get(region.getRegionNameAsString());
+    public PTableStatsImpl(TreeMap<byte[], List<byte[]>> guidePosts) {
+        this.guidePosts = guidePosts;
     }
 
     @Override
-    public Map<String, byte[][]> getGuidePosts(){
-      if(regionGuidePosts != null) {
-        return ImmutableMap.copyOf(regionGuidePosts);
-      }
-      
-      return null;
+    public TreeMap<byte[], List<byte[]>> getGuidePosts() {
+        return guidePosts;
     }
+
 }


[4/5] PHOENIX-180 Use stats to guide query parallelization (Ramkrishna S Vasudevan)

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableUpsertSelectIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableUpsertSelectIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableUpsertSelectIT.java
index e6fe7d3..690b15c 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableUpsertSelectIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableUpsertSelectIT.java
@@ -23,10 +23,12 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
+import java.io.IOException;
 import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
+import java.sql.SQLException;
 import java.util.Properties;
 
 import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
@@ -94,12 +96,12 @@ public class SaltedTableUpsertSelectIT extends BaseHBaseManagedTimeIT {
             stmt.setInt(2, 1);
             stmt.execute();
             conn.commit();
-            
             query = "UPSERT INTO target(pk, col) SELECT pk, col from source";
             stmt = conn.prepareStatement(query);
             stmt.execute();
             conn.commit();
-            
+            analyzeTable(conn, "source");
+            analyzeTable(conn, "target");
             query = "SELECT * FROM target";
             stmt = conn.prepareStatement(query);
             ResultSet rs = stmt.executeQuery();
@@ -111,6 +113,11 @@ public class SaltedTableUpsertSelectIT extends BaseHBaseManagedTimeIT {
             conn.close();
         }
     }
+    
+    private void analyzeTable(Connection conn, String tableName) throws IOException, SQLException {
+        String query = "ANALYZE " + tableName;
+        conn.createStatement().execute(query);
+    }
 
     @Test
     public void testUpsertSaltedTableIntoSaltedTable() throws Exception {
@@ -188,12 +195,12 @@ public class SaltedTableUpsertSelectIT extends BaseHBaseManagedTimeIT {
         Connection conn = DriverManager.getConnection(getUrl(), props);
         conn.setAutoCommit(false);
         try {
-            String ddl = "CREATE TABLE IF NOT EXISTS source" + 
+            String ddl = "CREATE TABLE IF NOT EXISTS source1" + 
                     " (pk1 varchar NULL, pk2 varchar NULL, pk3 integer NOT NULL, col1 INTEGER" + 
                     " CONSTRAINT pk PRIMARY KEY (pk1, pk2, pk3)) SALT_BUCKETS=4";
             createTestTable(getUrl(), ddl);
             
-            String query = "UPSERT INTO source(pk1, pk2, pk3, col1) VALUES(?,?,?,?)";
+            String query = "UPSERT INTO source1(pk1, pk2, pk3, col1) VALUES(?,?,?,?)";
             PreparedStatement stmt = conn.prepareStatement(query);
             stmt.setString(1, "1");
             stmt.setString(2, "2");
@@ -203,12 +210,12 @@ public class SaltedTableUpsertSelectIT extends BaseHBaseManagedTimeIT {
             conn.commit();
             
             conn.setAutoCommit(true);
-            query = "UPSERT INTO source(pk3, col1, pk1) SELECT pk3+1, col1+1, pk2 from source";
+            query = "UPSERT INTO source1(pk3, col1, pk1) SELECT pk3+1, col1+1, pk2 from source1";
             stmt = conn.prepareStatement(query);
             stmt.execute();
             conn.commit();
-            
-            query = "SELECT col1 FROM source";
+            analyzeTable(conn, "source1");
+            query = "SELECT col1 FROM source1";
             stmt = conn.prepareStatement(query);
             ResultSet rs = stmt.executeQuery();
             assertTrue(rs.next());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/main/antlr3/PhoenixSQL.g
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/antlr3/PhoenixSQL.g b/phoenix-core/src/main/antlr3/PhoenixSQL.g
index 556cd21..42b8a98 100644
--- a/phoenix-core/src/main/antlr3/PhoenixSQL.g
+++ b/phoenix-core/src/main/antlr3/PhoenixSQL.g
@@ -1,6 +1,4 @@
 /**
- * Copyright 2010 The Apache Software Foundation
- *
  * 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
@@ -105,12 +103,12 @@ tokens
     MAXVALUE='maxvalue';
     CYCLE='cycle';
     CASCADE='cascade';
+    ANALYZE='analyze';
 }
 
 
 @parser::header {
 /**
- * Copyright 2010 The Apache Software Foundation
  *
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -358,6 +356,7 @@ non_select_node returns [BindableStatement ret]
     |   s=alter_table_node
     |	s=create_sequence_node
     |	s=drop_sequence_node
+    |   s=update_statistics_node
     |   s=explain_node) { contextStack.pop();  $ret = s; }
     ;
     
@@ -494,6 +493,11 @@ alter_table_node returns [AlterTableStatement ret]
         { PTableType tt = v==null ? (QueryConstants.SYSTEM_SCHEMA_NAME.equals(t.getSchemaName()) ? PTableType.SYSTEM : PTableType.TABLE) : PTableType.VIEW; ret = ( c == null ? factory.addColumn(factory.namedTable(null,t), tt, d, ex!=null, p) : factory.dropColumn(factory.namedTable(null,t), tt, c, ex!=null) ); }
     ;
 
+update_statistics_node returns [UpdateStatisticsStatement ret]
+	:   ANALYZE t=from_table_name
+		{ret = factory.updateStatistics(factory.namedTable(null, t));}
+	;
+
 prop_name returns [String ret]
     :   p=identifier {$ret = SchemaUtil.normalizeIdentifier(p); }
     ;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/main/java/org/apache/phoenix/cache/GlobalCache.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/cache/GlobalCache.java b/phoenix-core/src/main/java/org/apache/phoenix/cache/GlobalCache.java
index a36db0b..fcef0ec 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/cache/GlobalCache.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/cache/GlobalCache.java
@@ -72,10 +72,13 @@ public class GlobalCache extends TenantCacheImpl {
             synchronized(this) {
                 result = metaDataCache;
                 if(result == null) {
+                    long maxTTL = Math.min(config.getLong(
+                            QueryServices.MAX_SERVER_METADATA_CACHE_TIME_TO_LIVE_MS_ATTRIB,
+                            QueryServicesOptions.DEFAULT_MAX_SERVER_METADATA_CACHE_TIME_TO_LIVE_MS), config.getLong(
+                            QueryServices.STATS_UPDATE_FREQ_MS_ATTRIB,
+                            QueryServicesOptions.DEFAULT_STATS_UPDATE_FREQ_MS));
                     long maxSize = config.getLong(QueryServices.MAX_SERVER_METADATA_CACHE_SIZE_ATTRIB,
                             QueryServicesOptions.DEFAULT_MAX_SERVER_METADATA_CACHE_SIZE);
-                    long maxTTL = config.getLong(QueryServices.MAX_SERVER_METADATA_CACHE_TIME_TO_LIVE_MS_ATTRIB,
-                            QueryServicesOptions.DEFAULT_MAX_SERVER_METADATA_CACHE_TIME_TO_LIVE_MS);
                     metaDataCache = result = CacheBuilder.newBuilder()
                             .maximumWeight(maxSize)
                             .expireAfterAccess(maxTTL, TimeUnit.MILLISECONDS)

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/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 2efedb3..65232ea 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
@@ -60,10 +60,12 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
+import java.util.TreeMap;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.HConstants;
@@ -71,6 +73,7 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.HTablePool;
 import org.apache.hadoop.hbase.client.Mutation;
@@ -95,6 +98,8 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.phoenix.cache.GlobalCache;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.AddColumnRequest;
+import org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest;
+import org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheRequest;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheResponse;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.CreateTableRequest;
@@ -128,8 +133,12 @@ import org.apache.phoenix.schema.PTable.LinkType;
 import org.apache.phoenix.schema.PTable.ViewType;
 import org.apache.phoenix.schema.PTableImpl;
 import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.schema.PhoenixArray;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.TableNotFoundException;
+import org.apache.phoenix.schema.stat.PTableStats;
+import org.apache.phoenix.schema.stat.PTableStatsImpl;
+import org.apache.phoenix.schema.stat.StatisticsUtils;
 import org.apache.phoenix.schema.tuple.ResultTuple;
 import org.apache.phoenix.trace.util.Tracing;
 import org.apache.phoenix.util.ByteUtil;
@@ -143,6 +152,7 @@ import org.slf4j.LoggerFactory;
 
 import com.google.common.cache.Cache;
 import com.google.common.collect.Lists;
+import com.google.protobuf.ByteString;
 import com.google.protobuf.RpcCallback;
 import com.google.protobuf.RpcController;
 import com.google.protobuf.Service;
@@ -162,6 +172,7 @@ import com.google.protobuf.Service;
  * 
  * @since 0.1
  */
+@SuppressWarnings("deprecation")
 public class MetaDataEndpointImpl extends MetaDataProtocol implements CoprocessorService, Coprocessor {
     private static final Logger logger = LoggerFactory.getLogger(MetaDataEndpointImpl.class);
 
@@ -276,7 +287,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     }
 
     private RegionCoprocessorEnvironment env;
-
+         
     private static final Log LOG = LogFactory.getLog(MetaDataEndpointImpl.class);
 
     /**
@@ -296,7 +307,6 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         } else {
             throw new CoprocessorException("Must be loaded on a table region!");
         }
-
         LOG.info("Starting Tracing-Metrics Systems");
         // Start the phoenix trace collection
         Tracing.addTraceMetricsSource();
@@ -454,7 +464,6 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         indexes.add(indexTable);
     }
 
-    @SuppressWarnings("deprecation")
     private void addColumnToTable(List<Cell> results, PName colName, PName famName,
         Cell[] colKeyValues, List<PColumn> columns, boolean isSalted) {
         int i = 0;
@@ -668,11 +677,89 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
               addColumnToTable(results, colName, famName, colKeyValues, columns, saltBucketNum != null);
           }
         }
-
+        PName physicalTableName = physicalTables.isEmpty() ? PNameFactory.newName(SchemaUtil.getTableName(
+                schemaName.getString(), tableName.getString())) : physicalTables.get(0);
+        PTableStats stats = updateStatsInternal(physicalTableName.getBytes(), columns);
         return PTableImpl.makePTable(tenantId, schemaName, tableName, tableType, indexState, timeStamp, 
             tableSeqNum, pkName, saltBucketNum, columns, tableType == INDEX ? dataTableName : null, 
             indexes, isImmutableRows, physicalTables, defaultFamilyName, viewStatement, disableWAL, 
-            multiTenant, viewType, viewIndexId, indexType);
+            multiTenant, viewType, viewIndexId, indexType, stats);
+    }
+
+    private PTableStats updateStatsInternal(byte[] tableNameBytes, List<PColumn> columns)
+            throws IOException {
+        List<PName> family = Lists.newArrayListWithExpectedSize(columns.size());
+        for (PColumn column : columns) {
+            PName familyName = column.getFamilyName();
+            if (familyName != null) {
+                family.add(familyName);
+            }
+        }
+        HTable statsHTable = null;
+        try {
+            // Can we do a new HTable instance here? Or get it from a pool or cache of these instances?
+            statsHTable = new HTable(this.env.getConfiguration(),
+                    PhoenixDatabaseMetaData.SYSTEM_STATS_NAME_BYTES);
+            Scan s = new Scan();
+            if (tableNameBytes != null) {
+                // Check for an efficient way here
+                s.setStartRow(tableNameBytes);
+                s.setStopRow(ByteUtil.nextKey(tableNameBytes));
+            }
+            ResultScanner scanner = statsHTable.getScanner(s);
+            Result result = null;
+            byte[] fam = null;
+            List<byte[]> guidePosts = Lists.newArrayListWithExpectedSize(columns.size());
+            TreeMap<byte[], List<byte[]>> guidePostsPerCf = new TreeMap<byte[], List<byte[]>>(Bytes.BYTES_COMPARATOR);
+            while ((result = scanner.next()) != null) {
+                CellScanner cellScanner = result.cellScanner();
+                while (cellScanner.advance()) {
+                    Cell current = cellScanner.current();
+                    // For now collect only guide posts
+                    if (Bytes.equals(current.getQualifierArray(), current.getQualifierOffset(),
+                            current.getQualifierLength(), PhoenixDatabaseMetaData.GUIDE_POSTS_BYTES, 0,
+                            PhoenixDatabaseMetaData.GUIDE_POSTS_BYTES.length)) {
+                        byte[] cfInCell = StatisticsUtils.getCFFromRowKey(tableNameBytes, current.getRowArray(),
+                                current.getRowOffset(), current.getRowLength());
+                        if (fam == null) {
+                            fam = cfInCell;
+                        } else if (!Bytes.equals(fam, cfInCell)) {
+                            // Sort all the guide posts
+                            guidePostsPerCf.put(cfInCell, guidePosts);
+                            guidePosts = new ArrayList<byte[]>();
+                            fam = cfInCell;
+                        }
+                        byte[] guidePostVal = new ImmutableBytesPtr(current.getValueArray(), current.getValueOffset(), current
+                                .getValueLength()).copyBytesIfNecessary();
+                        PhoenixArray array = (PhoenixArray)PDataType.VARBINARY_ARRAY.toObject(guidePostVal);
+                        if (array != null && array.getDimensions() != 0) {
+                            for (int j = 0; j < array.getDimensions(); j++) {
+                                byte[] gp = array.toBytes(j);
+                                if (gp.length != 0) {
+                                    guidePosts.add(gp);
+                                }
+                            }
+                        }
+                    }
+                }
+            }
+            if(fam != null) {
+                // Sort all the guideposts
+                guidePostsPerCf.put(fam, guidePosts);
+            }
+            return new PTableStatsImpl(guidePostsPerCf);
+        } catch (Exception e) {
+            if (e instanceof org.apache.hadoop.hbase.TableNotFoundException) {
+                logger.warn("Stats table not yet online", e);
+            } else {
+                throw new IOException(e);
+            }
+        } finally {
+            if (statsHTable != null) {
+                statsHTable.close();
+            }
+        }
+        return PTableStatsImpl.NO_STATS;
     }
 
     private PTable buildDeletedTable(byte[] key, ImmutableBytesPtr cacheKey, HRegion region,
@@ -866,7 +953,6 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
      * Looks for whether child views exist for the table specified by table.
      * TODO: should we pass a timestamp here?
      */
-    @SuppressWarnings("deprecation")
     private TableViewFinderResult findChildViews(HRegion region, byte[] tenantId, PTable table) throws IOException {
         byte[] schemaName = table.getSchemaName().getBytes();
         byte[] tableName = table.getTableName().getBytes();
@@ -1464,7 +1550,6 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         done.run(builder.build());
     }
 
-    @SuppressWarnings("deprecation")
     @Override
     public void updateIndexState(RpcController controller, UpdateIndexStateRequest request,
             RpcCallback<MetaDataResponse> done) {
@@ -1672,4 +1757,37 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         }
     }
     
+    @Override
+    public void clearCacheForTable(RpcController controller, ClearCacheForTableRequest request,
+            RpcCallback<ClearCacheForTableResponse> done) {
+        ByteString tenantId = request.getTenantId();
+        ByteString schemaName = request.getSchemaName();
+        ByteString tableName = request.getTableName();
+        byte[] tableKey = SchemaUtil.getTableKey(tenantId.toByteArray(), schemaName.toByteArray(),
+                tableName.toByteArray());
+        ImmutableBytesPtr key = new ImmutableBytesPtr(tableKey);
+        try {
+            PTable table = doGetTable(tableKey, request.getClientTimestamp());
+            if (table != null) {
+                Cache<ImmutableBytesPtr, PTable> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
+                // Add +1 to the ts
+                // TODO : refactor doGetTable() to do this - optionally increment the timestamp
+                // TODO : clear metadata if it is spread across multiple region servers
+                long ts = table.getTimeStamp() + 1;
+                // Here we could add an empty puti
+                HRegion region = env.getRegion();
+                List<Mutation> mutations = new ArrayList<Mutation>();
+                Put p = new Put(tableKey);
+                p.add(TABLE_FAMILY_BYTES, QueryConstants.EMPTY_COLUMN_BYTES, ts, ByteUtil.EMPTY_BYTE_ARRAY);
+                mutations.add(p);
+                region.mutateRowsWithLocks(mutations, Collections.<byte[]> emptySet());
+                metaDataCache.invalidate(key);
+            }
+        } catch (Throwable t) {
+            // We could still invalidate it
+            logger.error("clearCacheForTable failed to update the latest ts ", t);
+            ProtobufUtil.setControllerException(controller, ServerUtil.createIOException(
+                    SchemaUtil.getTableName(schemaName.toString(), tableName.toString()), t));
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/MetaDataProtos.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/MetaDataProtos.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/MetaDataProtos.java
index 7877fc4..5ac9cb7 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/MetaDataProtos.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/MetaDataProtos.java
@@ -6316,6 +6316,1089 @@ public final class MetaDataProtos {
     // @@protoc_insertion_point(class_scope:GetVersionResponse)
   }
 
+  public interface ClearCacheForTableRequestOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // required bytes tenantId = 1;
+    /**
+     * <code>required bytes tenantId = 1;</code>
+     */
+    boolean hasTenantId();
+    /**
+     * <code>required bytes tenantId = 1;</code>
+     */
+    com.google.protobuf.ByteString getTenantId();
+
+    // required bytes schemaName = 2;
+    /**
+     * <code>required bytes schemaName = 2;</code>
+     */
+    boolean hasSchemaName();
+    /**
+     * <code>required bytes schemaName = 2;</code>
+     */
+    com.google.protobuf.ByteString getSchemaName();
+
+    // required bytes tableName = 3;
+    /**
+     * <code>required bytes tableName = 3;</code>
+     */
+    boolean hasTableName();
+    /**
+     * <code>required bytes tableName = 3;</code>
+     */
+    com.google.protobuf.ByteString getTableName();
+
+    // required int64 clientTimestamp = 4;
+    /**
+     * <code>required int64 clientTimestamp = 4;</code>
+     */
+    boolean hasClientTimestamp();
+    /**
+     * <code>required int64 clientTimestamp = 4;</code>
+     */
+    long getClientTimestamp();
+  }
+  /**
+   * Protobuf type {@code ClearCacheForTableRequest}
+   */
+  public static final class ClearCacheForTableRequest extends
+      com.google.protobuf.GeneratedMessage
+      implements ClearCacheForTableRequestOrBuilder {
+    // Use ClearCacheForTableRequest.newBuilder() to construct.
+    private ClearCacheForTableRequest(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private ClearCacheForTableRequest(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final ClearCacheForTableRequest defaultInstance;
+    public static ClearCacheForTableRequest getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public ClearCacheForTableRequest getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private ClearCacheForTableRequest(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              bitField0_ |= 0x00000001;
+              tenantId_ = input.readBytes();
+              break;
+            }
+            case 18: {
+              bitField0_ |= 0x00000002;
+              schemaName_ = input.readBytes();
+              break;
+            }
+            case 26: {
+              bitField0_ |= 0x00000004;
+              tableName_ = input.readBytes();
+              break;
+            }
+            case 32: {
+              bitField0_ |= 0x00000008;
+              clientTimestamp_ = input.readInt64();
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_ClearCacheForTableRequest_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_ClearCacheForTableRequest_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest.class, org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<ClearCacheForTableRequest> PARSER =
+        new com.google.protobuf.AbstractParser<ClearCacheForTableRequest>() {
+      public ClearCacheForTableRequest parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new ClearCacheForTableRequest(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<ClearCacheForTableRequest> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // required bytes tenantId = 1;
+    public static final int TENANTID_FIELD_NUMBER = 1;
+    private com.google.protobuf.ByteString tenantId_;
+    /**
+     * <code>required bytes tenantId = 1;</code>
+     */
+    public boolean hasTenantId() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required bytes tenantId = 1;</code>
+     */
+    public com.google.protobuf.ByteString getTenantId() {
+      return tenantId_;
+    }
+
+    // required bytes schemaName = 2;
+    public static final int SCHEMANAME_FIELD_NUMBER = 2;
+    private com.google.protobuf.ByteString schemaName_;
+    /**
+     * <code>required bytes schemaName = 2;</code>
+     */
+    public boolean hasSchemaName() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>required bytes schemaName = 2;</code>
+     */
+    public com.google.protobuf.ByteString getSchemaName() {
+      return schemaName_;
+    }
+
+    // required bytes tableName = 3;
+    public static final int TABLENAME_FIELD_NUMBER = 3;
+    private com.google.protobuf.ByteString tableName_;
+    /**
+     * <code>required bytes tableName = 3;</code>
+     */
+    public boolean hasTableName() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>required bytes tableName = 3;</code>
+     */
+    public com.google.protobuf.ByteString getTableName() {
+      return tableName_;
+    }
+
+    // required int64 clientTimestamp = 4;
+    public static final int CLIENTTIMESTAMP_FIELD_NUMBER = 4;
+    private long clientTimestamp_;
+    /**
+     * <code>required int64 clientTimestamp = 4;</code>
+     */
+    public boolean hasClientTimestamp() {
+      return ((bitField0_ & 0x00000008) == 0x00000008);
+    }
+    /**
+     * <code>required int64 clientTimestamp = 4;</code>
+     */
+    public long getClientTimestamp() {
+      return clientTimestamp_;
+    }
+
+    private void initFields() {
+      tenantId_ = com.google.protobuf.ByteString.EMPTY;
+      schemaName_ = com.google.protobuf.ByteString.EMPTY;
+      tableName_ = com.google.protobuf.ByteString.EMPTY;
+      clientTimestamp_ = 0L;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (!hasTenantId()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasSchemaName()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasTableName()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasClientTimestamp()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeBytes(1, tenantId_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeBytes(2, schemaName_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeBytes(3, tableName_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        output.writeInt64(4, clientTimestamp_);
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(1, tenantId_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(2, schemaName_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(3, tableName_);
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt64Size(4, clientTimestamp_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest)) {
+        return super.equals(obj);
+      }
+      org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest other = (org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest) obj;
+
+      boolean result = true;
+      result = result && (hasTenantId() == other.hasTenantId());
+      if (hasTenantId()) {
+        result = result && getTenantId()
+            .equals(other.getTenantId());
+      }
+      result = result && (hasSchemaName() == other.hasSchemaName());
+      if (hasSchemaName()) {
+        result = result && getSchemaName()
+            .equals(other.getSchemaName());
+      }
+      result = result && (hasTableName() == other.hasTableName());
+      if (hasTableName()) {
+        result = result && getTableName()
+            .equals(other.getTableName());
+      }
+      result = result && (hasClientTimestamp() == other.hasClientTimestamp());
+      if (hasClientTimestamp()) {
+        result = result && (getClientTimestamp()
+            == other.getClientTimestamp());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasTenantId()) {
+        hash = (37 * hash) + TENANTID_FIELD_NUMBER;
+        hash = (53 * hash) + getTenantId().hashCode();
+      }
+      if (hasSchemaName()) {
+        hash = (37 * hash) + SCHEMANAME_FIELD_NUMBER;
+        hash = (53 * hash) + getSchemaName().hashCode();
+      }
+      if (hasTableName()) {
+        hash = (37 * hash) + TABLENAME_FIELD_NUMBER;
+        hash = (53 * hash) + getTableName().hashCode();
+      }
+      if (hasClientTimestamp()) {
+        hash = (37 * hash) + CLIENTTIMESTAMP_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getClientTimestamp());
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code ClearCacheForTableRequest}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequestOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_ClearCacheForTableRequest_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_ClearCacheForTableRequest_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest.class, org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest.Builder.class);
+      }
+
+      // Construct using org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        tenantId_ = com.google.protobuf.ByteString.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        schemaName_ = com.google.protobuf.ByteString.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        tableName_ = com.google.protobuf.ByteString.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000004);
+        clientTimestamp_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000008);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_ClearCacheForTableRequest_descriptor;
+      }
+
+      public org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest getDefaultInstanceForType() {
+        return org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest.getDefaultInstance();
+      }
+
+      public org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest build() {
+        org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest buildPartial() {
+        org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest result = new org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.tenantId_ = tenantId_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.schemaName_ = schemaName_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        result.tableName_ = tableName_;
+        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+          to_bitField0_ |= 0x00000008;
+        }
+        result.clientTimestamp_ = clientTimestamp_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest) {
+          return mergeFrom((org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest other) {
+        if (other == org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest.getDefaultInstance()) return this;
+        if (other.hasTenantId()) {
+          setTenantId(other.getTenantId());
+        }
+        if (other.hasSchemaName()) {
+          setSchemaName(other.getSchemaName());
+        }
+        if (other.hasTableName()) {
+          setTableName(other.getTableName());
+        }
+        if (other.hasClientTimestamp()) {
+          setClientTimestamp(other.getClientTimestamp());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasTenantId()) {
+          
+          return false;
+        }
+        if (!hasSchemaName()) {
+          
+          return false;
+        }
+        if (!hasTableName()) {
+          
+          return false;
+        }
+        if (!hasClientTimestamp()) {
+          
+          return false;
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // required bytes tenantId = 1;
+      private com.google.protobuf.ByteString tenantId_ = com.google.protobuf.ByteString.EMPTY;
+      /**
+       * <code>required bytes tenantId = 1;</code>
+       */
+      public boolean hasTenantId() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required bytes tenantId = 1;</code>
+       */
+      public com.google.protobuf.ByteString getTenantId() {
+        return tenantId_;
+      }
+      /**
+       * <code>required bytes tenantId = 1;</code>
+       */
+      public Builder setTenantId(com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        tenantId_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required bytes tenantId = 1;</code>
+       */
+      public Builder clearTenantId() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        tenantId_ = getDefaultInstance().getTenantId();
+        onChanged();
+        return this;
+      }
+
+      // required bytes schemaName = 2;
+      private com.google.protobuf.ByteString schemaName_ = com.google.protobuf.ByteString.EMPTY;
+      /**
+       * <code>required bytes schemaName = 2;</code>
+       */
+      public boolean hasSchemaName() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>required bytes schemaName = 2;</code>
+       */
+      public com.google.protobuf.ByteString getSchemaName() {
+        return schemaName_;
+      }
+      /**
+       * <code>required bytes schemaName = 2;</code>
+       */
+      public Builder setSchemaName(com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+        schemaName_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required bytes schemaName = 2;</code>
+       */
+      public Builder clearSchemaName() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        schemaName_ = getDefaultInstance().getSchemaName();
+        onChanged();
+        return this;
+      }
+
+      // required bytes tableName = 3;
+      private com.google.protobuf.ByteString tableName_ = com.google.protobuf.ByteString.EMPTY;
+      /**
+       * <code>required bytes tableName = 3;</code>
+       */
+      public boolean hasTableName() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <code>required bytes tableName = 3;</code>
+       */
+      public com.google.protobuf.ByteString getTableName() {
+        return tableName_;
+      }
+      /**
+       * <code>required bytes tableName = 3;</code>
+       */
+      public Builder setTableName(com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000004;
+        tableName_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required bytes tableName = 3;</code>
+       */
+      public Builder clearTableName() {
+        bitField0_ = (bitField0_ & ~0x00000004);
+        tableName_ = getDefaultInstance().getTableName();
+        onChanged();
+        return this;
+      }
+
+      // required int64 clientTimestamp = 4;
+      private long clientTimestamp_ ;
+      /**
+       * <code>required int64 clientTimestamp = 4;</code>
+       */
+      public boolean hasClientTimestamp() {
+        return ((bitField0_ & 0x00000008) == 0x00000008);
+      }
+      /**
+       * <code>required int64 clientTimestamp = 4;</code>
+       */
+      public long getClientTimestamp() {
+        return clientTimestamp_;
+      }
+      /**
+       * <code>required int64 clientTimestamp = 4;</code>
+       */
+      public Builder setClientTimestamp(long value) {
+        bitField0_ |= 0x00000008;
+        clientTimestamp_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required int64 clientTimestamp = 4;</code>
+       */
+      public Builder clearClientTimestamp() {
+        bitField0_ = (bitField0_ & ~0x00000008);
+        clientTimestamp_ = 0L;
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:ClearCacheForTableRequest)
+    }
+
+    static {
+      defaultInstance = new ClearCacheForTableRequest(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:ClearCacheForTableRequest)
+  }
+
+  public interface ClearCacheForTableResponseOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+  }
+  /**
+   * Protobuf type {@code ClearCacheForTableResponse}
+   */
+  public static final class ClearCacheForTableResponse extends
+      com.google.protobuf.GeneratedMessage
+      implements ClearCacheForTableResponseOrBuilder {
+    // Use ClearCacheForTableResponse.newBuilder() to construct.
+    private ClearCacheForTableResponse(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private ClearCacheForTableResponse(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final ClearCacheForTableResponse defaultInstance;
+    public static ClearCacheForTableResponse getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public ClearCacheForTableResponse getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private ClearCacheForTableResponse(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_ClearCacheForTableResponse_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_ClearCacheForTableResponse_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse.class, org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<ClearCacheForTableResponse> PARSER =
+        new com.google.protobuf.AbstractParser<ClearCacheForTableResponse>() {
+      public ClearCacheForTableResponse parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new ClearCacheForTableResponse(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<ClearCacheForTableResponse> getParserForType() {
+      return PARSER;
+    }
+
+    private void initFields() {
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse)) {
+        return super.equals(obj);
+      }
+      org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse other = (org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse) obj;
+
+      boolean result = true;
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code ClearCacheForTableResponse}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponseOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_ClearCacheForTableResponse_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_ClearCacheForTableResponse_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse.class, org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse.Builder.class);
+      }
+
+      // Construct using org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.phoenix.coprocessor.generated.MetaDataProtos.internal_static_ClearCacheForTableResponse_descriptor;
+      }
+
+      public org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse getDefaultInstanceForType() {
+        return org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse.getDefaultInstance();
+      }
+
+      public org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse build() {
+        org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse buildPartial() {
+        org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse result = new org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse(this);
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse) {
+          return mergeFrom((org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse other) {
+        if (other == org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse.getDefaultInstance()) return this;
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:ClearCacheForTableResponse)
+    }
+
+    static {
+      defaultInstance = new ClearCacheForTableResponse(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:ClearCacheForTableResponse)
+  }
+
   /**
    * Protobuf service {@code MetaDataService}
    */
@@ -6388,6 +7471,14 @@ public final class MetaDataProtos {
           org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionRequest request,
           com.google.protobuf.RpcCallback<org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionResponse> done);
 
+      /**
+       * <code>rpc clearCacheForTable(.ClearCacheForTableRequest) returns (.ClearCacheForTableResponse);</code>
+       */
+      public abstract void clearCacheForTable(
+          com.google.protobuf.RpcController controller,
+          org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest request,
+          com.google.protobuf.RpcCallback<org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse> done);
+
     }
 
     public static com.google.protobuf.Service newReflectiveService(
@@ -6457,6 +7548,14 @@ public final class MetaDataProtos {
           impl.getVersion(controller, request, done);
         }
 
+        @java.lang.Override
+        public  void clearCacheForTable(
+            com.google.protobuf.RpcController controller,
+            org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest request,
+            com.google.protobuf.RpcCallback<org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse> done) {
+          impl.clearCacheForTable(controller, request, done);
+        }
+
       };
     }
 
@@ -6495,6 +7594,8 @@ public final class MetaDataProtos {
               return impl.clearCache(controller, (org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheRequest)request);
             case 7:
               return impl.getVersion(controller, (org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionRequest)request);
+            case 8:
+              return impl.clearCacheForTable(controller, (org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest)request);
             default:
               throw new java.lang.AssertionError("Can't get here.");
           }
@@ -6525,6 +7626,8 @@ public final class MetaDataProtos {
               return org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheRequest.getDefaultInstance();
             case 7:
               return org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionRequest.getDefaultInstance();
+            case 8:
+              return org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest.getDefaultInstance();
             default:
               throw new java.lang.AssertionError("Can't get here.");
           }
@@ -6555,6 +7658,8 @@ public final class MetaDataProtos {
               return org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheResponse.getDefaultInstance();
             case 7:
               return org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionResponse.getDefaultInstance();
+            case 8:
+              return org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse.getDefaultInstance();
             default:
               throw new java.lang.AssertionError("Can't get here.");
           }
@@ -6627,6 +7732,14 @@ public final class MetaDataProtos {
         org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionRequest request,
         com.google.protobuf.RpcCallback<org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionResponse> done);
 
+    /**
+     * <code>rpc clearCacheForTable(.ClearCacheForTableRequest) returns (.ClearCacheForTableResponse);</code>
+     */
+    public abstract void clearCacheForTable(
+        com.google.protobuf.RpcController controller,
+        org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest request,
+        com.google.protobuf.RpcCallback<org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse> done);
+
     public static final
         com.google.protobuf.Descriptors.ServiceDescriptor
         getDescriptor() {
@@ -6689,6 +7802,11 @@ public final class MetaDataProtos {
             com.google.protobuf.RpcUtil.<org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionResponse>specializeCallback(
               done));
           return;
+        case 8:
+          this.clearCacheForTable(controller, (org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest)request,
+            com.google.protobuf.RpcUtil.<org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse>specializeCallback(
+              done));
+          return;
         default:
           throw new java.lang.AssertionError("Can't get here.");
       }
@@ -6719,6 +7837,8 @@ public final class MetaDataProtos {
           return org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheRequest.getDefaultInstance();
         case 7:
           return org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionRequest.getDefaultInstance();
+        case 8:
+          return org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest.getDefaultInstance();
         default:
           throw new java.lang.AssertionError("Can't get here.");
       }
@@ -6749,6 +7869,8 @@ public final class MetaDataProtos {
           return org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheResponse.getDefaultInstance();
         case 7:
           return org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionResponse.getDefaultInstance();
+        case 8:
+          return org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse.getDefaultInstance();
         default:
           throw new java.lang.AssertionError("Can't get here.");
       }
@@ -6889,6 +8011,21 @@ public final class MetaDataProtos {
             org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionResponse.class,
             org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionResponse.getDefaultInstance()));
       }
+
+      public  void clearCacheForTable(
+          com.google.protobuf.RpcController controller,
+          org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest request,
+          com.google.protobuf.RpcCallback<org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(8),
+          controller,
+          request,
+          org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse.class,
+            org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse.getDefaultInstance()));
+      }
     }
 
     public static BlockingInterface newBlockingStub(
@@ -6936,6 +8073,11 @@ public final class MetaDataProtos {
           com.google.protobuf.RpcController controller,
           org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionRequest request)
           throws com.google.protobuf.ServiceException;
+
+      public org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse clearCacheForTable(
+          com.google.protobuf.RpcController controller,
+          org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest request)
+          throws com.google.protobuf.ServiceException;
     }
 
     private static final class BlockingStub implements BlockingInterface {
@@ -7040,6 +8182,18 @@ public final class MetaDataProtos {
           org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionResponse.getDefaultInstance());
       }
 
+
+      public org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse clearCacheForTable(
+          com.google.protobuf.RpcController controller,
+          org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableRequest request)
+          throws com.google.protobuf.ServiceException {
+        return (org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse) channel.callBlockingMethod(
+          getDescriptor().getMethods().get(8),
+          controller,
+          request,
+          org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheForTableResponse.getDefaultInstance());
+      }
+
     }
 
     // @@protoc_insertion_point(class_scope:MetaDataService)
@@ -7100,6 +8254,16 @@ public final class MetaDataProtos {
   private static
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
       internal_static_GetVersionResponse_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_ClearCacheForTableRequest_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_ClearCacheForTableRequest_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_ClearCacheForTableResponse_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_ClearCacheForTableResponse_fieldAccessorTable;
 
   public static com.google.protobuf.Descriptors.FileDescriptor
       getDescriptor() {
@@ -7127,27 +8291,32 @@ public final class MetaDataProtos {
       "st\022\036\n\026tableMetadataMutations\030\001 \003(\014\"\023\n\021Cl" +
       "earCacheRequest\"\024\n\022ClearCacheResponse\"\023\n" +
       "\021GetVersionRequest\"%\n\022GetVersionResponse" +
-      "\022\017\n\007version\030\001 \002(\003*\212\002\n\014MutationCode\022\030\n\024TA" +
-      "BLE_ALREADY_EXISTS\020\000\022\023\n\017TABLE_NOT_FOUND\020",
-      "\001\022\024\n\020COLUMN_NOT_FOUND\020\002\022\031\n\025COLUMN_ALREAD" +
-      "Y_EXISTS\020\003\022\035\n\031CONCURRENT_TABLE_MUTATION\020" +
-      "\004\022\027\n\023TABLE_NOT_IN_REGION\020\005\022\025\n\021NEWER_TABL" +
-      "E_FOUND\020\006\022\034\n\030UNALLOWED_TABLE_MUTATION\020\007\022" +
-      "\021\n\rNO_PK_COLUMNS\020\010\022\032\n\026PARENT_TABLE_NOT_F" +
-      "OUND\020\t2\303\003\n\017MetaDataService\022/\n\010getTable\022\020" +
-      ".GetTableRequest\032\021.MetaDataResponse\0225\n\013c" +
-      "reateTable\022\023.CreateTableRequest\032\021.MetaDa" +
-      "taResponse\0221\n\tdropTable\022\021.DropTableReque" +
-      "st\032\021.MetaDataResponse\0221\n\taddColumn\022\021.Add",
-      "ColumnRequest\032\021.MetaDataResponse\0223\n\ndrop" +
-      "Column\022\022.DropColumnRequest\032\021.MetaDataRes" +
-      "ponse\022?\n\020updateIndexState\022\030.UpdateIndexS" +
-      "tateRequest\032\021.MetaDataResponse\0225\n\nclearC" +
-      "ache\022\022.ClearCacheRequest\032\023.ClearCacheRes" +
-      "ponse\0225\n\ngetVersion\022\022.GetVersionRequest\032" +
-      "\023.GetVersionResponseBB\n(org.apache.phoen" +
-      "ix.coprocessor.generatedB\016MetaDataProtos" +
-      "H\001\210\001\001\240\001\001"
+      "\022\017\n\007version\030\001 \002(\003\"m\n\031ClearCacheForTableR" +
+      "equest\022\020\n\010tenantId\030\001 \002(\014\022\022\n\nschemaName\030\002",
+      " \002(\014\022\021\n\ttableName\030\003 \002(\014\022\027\n\017clientTimesta" +
+      "mp\030\004 \002(\003\"\034\n\032ClearCacheForTableResponse*\212" +
+      "\002\n\014MutationCode\022\030\n\024TABLE_ALREADY_EXISTS\020" +
+      "\000\022\023\n\017TABLE_NOT_FOUND\020\001\022\024\n\020COLUMN_NOT_FOU" +
+      "ND\020\002\022\031\n\025COLUMN_ALREADY_EXISTS\020\003\022\035\n\031CONCU" +
+      "RRENT_TABLE_MUTATION\020\004\022\027\n\023TABLE_NOT_IN_R" +
+      "EGION\020\005\022\025\n\021NEWER_TABLE_FOUND\020\006\022\034\n\030UNALLO" +
+      "WED_TABLE_MUTATION\020\007\022\021\n\rNO_PK_COLUMNS\020\010\022" +
+      "\032\n\026PARENT_TABLE_NOT_FOUND\020\t2\222\004\n\017MetaData" +
+      "Service\022/\n\010getTable\022\020.GetTableRequest\032\021.",
+      "MetaDataResponse\0225\n\013createTable\022\023.Create" +
+      "TableRequest\032\021.MetaDataResponse\0221\n\tdropT" +
+      "able\022\021.DropTableRequest\032\021.MetaDataRespon" +
+      "se\0221\n\taddColumn\022\021.AddColumnRequest\032\021.Met" +
+      "aDataResponse\0223\n\ndropColumn\022\022.DropColumn" +
+      "Request\032\021.MetaDataResponse\022?\n\020updateInde" +
+      "xState\022\030.UpdateIndexStateRequest\032\021.MetaD" +
+      "ataResponse\0225\n\nclearCache\022\022.ClearCacheRe" +
+      "quest\032\023.ClearCacheResponse\0225\n\ngetVersion" +
+      "\022\022.GetVersionRequest\032\023.GetVersionRespons",
+      "e\022M\n\022clearCacheForTable\022\032.ClearCacheForT" +
+      "ableRequest\032\033.ClearCacheForTableResponse" +
+      "BB\n(org.apache.phoenix.coprocessor.gener" +
+      "atedB\016MetaDataProtosH\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -7220,6 +8389,18 @@ public final class MetaDataProtos {
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_GetVersionResponse_descriptor,
               new java.lang.String[] { "Version", });
+          internal_static_ClearCacheForTableRequest_descriptor =
+            getDescriptor().getMessageTypes().get(11);
+          internal_static_ClearCacheForTableRequest_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_ClearCacheForTableRequest_descriptor,
+              new java.lang.String[] { "TenantId", "SchemaName", "TableName", "ClientTimestamp", });
+          internal_static_ClearCacheForTableResponse_descriptor =
+            getDescriptor().getMessageTypes().get(12);
+          internal_static_ClearCacheForTableResponse_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_ClearCacheForTableResponse_descriptor,
+              new java.lang.String[] { });
           return null;
         }
       };


[3/5] PHOENIX-180 Use stats to guide query parallelization (Ramkrishna S Vasudevan)

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/StatCollectorProtos.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/StatCollectorProtos.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/StatCollectorProtos.java
new file mode 100644
index 0000000..5008f22
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/StatCollectorProtos.java
@@ -0,0 +1,1269 @@
+// Generated by the protocol buffer compiler.  DO NOT EDIT!
+// source: StatisticsCollect.proto
+
+package org.apache.phoenix.coprocessor.generated;
+
+public final class StatCollectorProtos {
+  private StatCollectorProtos() {}
+  public static void registerAllExtensions(
+      com.google.protobuf.ExtensionRegistry registry) {
+  }
+  public interface StatCollectRequestOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // optional bytes startRow = 1;
+    /**
+     * <code>optional bytes startRow = 1;</code>
+     */
+    boolean hasStartRow();
+    /**
+     * <code>optional bytes startRow = 1;</code>
+     */
+    com.google.protobuf.ByteString getStartRow();
+
+    // optional bytes stopRow = 2;
+    /**
+     * <code>optional bytes stopRow = 2;</code>
+     */
+    boolean hasStopRow();
+    /**
+     * <code>optional bytes stopRow = 2;</code>
+     */
+    com.google.protobuf.ByteString getStopRow();
+  }
+  /**
+   * Protobuf type {@code StatCollectRequest}
+   */
+  public static final class StatCollectRequest extends
+      com.google.protobuf.GeneratedMessage
+      implements StatCollectRequestOrBuilder {
+    // Use StatCollectRequest.newBuilder() to construct.
+    private StatCollectRequest(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private StatCollectRequest(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final StatCollectRequest defaultInstance;
+    public static StatCollectRequest getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public StatCollectRequest getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private StatCollectRequest(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              bitField0_ |= 0x00000001;
+              startRow_ = input.readBytes();
+              break;
+            }
+            case 18: {
+              bitField0_ |= 0x00000002;
+              stopRow_ = input.readBytes();
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.phoenix.coprocessor.generated.StatCollectorProtos.internal_static_StatCollectRequest_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.phoenix.coprocessor.generated.StatCollectorProtos.internal_static_StatCollectRequest_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest.class, org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<StatCollectRequest> PARSER =
+        new com.google.protobuf.AbstractParser<StatCollectRequest>() {
+      public StatCollectRequest parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new StatCollectRequest(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<StatCollectRequest> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // optional bytes startRow = 1;
+    public static final int STARTROW_FIELD_NUMBER = 1;
+    private com.google.protobuf.ByteString startRow_;
+    /**
+     * <code>optional bytes startRow = 1;</code>
+     */
+    public boolean hasStartRow() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>optional bytes startRow = 1;</code>
+     */
+    public com.google.protobuf.ByteString getStartRow() {
+      return startRow_;
+    }
+
+    // optional bytes stopRow = 2;
+    public static final int STOPROW_FIELD_NUMBER = 2;
+    private com.google.protobuf.ByteString stopRow_;
+    /**
+     * <code>optional bytes stopRow = 2;</code>
+     */
+    public boolean hasStopRow() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>optional bytes stopRow = 2;</code>
+     */
+    public com.google.protobuf.ByteString getStopRow() {
+      return stopRow_;
+    }
+
+    private void initFields() {
+      startRow_ = com.google.protobuf.ByteString.EMPTY;
+      stopRow_ = com.google.protobuf.ByteString.EMPTY;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeBytes(1, startRow_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeBytes(2, stopRow_);
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(1, startRow_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(2, stopRow_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest)) {
+        return super.equals(obj);
+      }
+      org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest other = (org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest) obj;
+
+      boolean result = true;
+      result = result && (hasStartRow() == other.hasStartRow());
+      if (hasStartRow()) {
+        result = result && getStartRow()
+            .equals(other.getStartRow());
+      }
+      result = result && (hasStopRow() == other.hasStopRow());
+      if (hasStopRow()) {
+        result = result && getStopRow()
+            .equals(other.getStopRow());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasStartRow()) {
+        hash = (37 * hash) + STARTROW_FIELD_NUMBER;
+        hash = (53 * hash) + getStartRow().hashCode();
+      }
+      if (hasStopRow()) {
+        hash = (37 * hash) + STOPROW_FIELD_NUMBER;
+        hash = (53 * hash) + getStopRow().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code StatCollectRequest}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequestOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.phoenix.coprocessor.generated.StatCollectorProtos.internal_static_StatCollectRequest_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.phoenix.coprocessor.generated.StatCollectorProtos.internal_static_StatCollectRequest_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest.class, org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest.Builder.class);
+      }
+
+      // Construct using org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        startRow_ = com.google.protobuf.ByteString.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        stopRow_ = com.google.protobuf.ByteString.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.phoenix.coprocessor.generated.StatCollectorProtos.internal_static_StatCollectRequest_descriptor;
+      }
+
+      public org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest getDefaultInstanceForType() {
+        return org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest.getDefaultInstance();
+      }
+
+      public org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest build() {
+        org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest buildPartial() {
+        org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest result = new org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.startRow_ = startRow_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.stopRow_ = stopRow_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest) {
+          return mergeFrom((org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest other) {
+        if (other == org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest.getDefaultInstance()) return this;
+        if (other.hasStartRow()) {
+          setStartRow(other.getStartRow());
+        }
+        if (other.hasStopRow()) {
+          setStopRow(other.getStopRow());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // optional bytes startRow = 1;
+      private com.google.protobuf.ByteString startRow_ = com.google.protobuf.ByteString.EMPTY;
+      /**
+       * <code>optional bytes startRow = 1;</code>
+       */
+      public boolean hasStartRow() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>optional bytes startRow = 1;</code>
+       */
+      public com.google.protobuf.ByteString getStartRow() {
+        return startRow_;
+      }
+      /**
+       * <code>optional bytes startRow = 1;</code>
+       */
+      public Builder setStartRow(com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        startRow_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bytes startRow = 1;</code>
+       */
+      public Builder clearStartRow() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        startRow_ = getDefaultInstance().getStartRow();
+        onChanged();
+        return this;
+      }
+
+      // optional bytes stopRow = 2;
+      private com.google.protobuf.ByteString stopRow_ = com.google.protobuf.ByteString.EMPTY;
+      /**
+       * <code>optional bytes stopRow = 2;</code>
+       */
+      public boolean hasStopRow() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>optional bytes stopRow = 2;</code>
+       */
+      public com.google.protobuf.ByteString getStopRow() {
+        return stopRow_;
+      }
+      /**
+       * <code>optional bytes stopRow = 2;</code>
+       */
+      public Builder setStopRow(com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+        stopRow_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bytes stopRow = 2;</code>
+       */
+      public Builder clearStopRow() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        stopRow_ = getDefaultInstance().getStopRow();
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:StatCollectRequest)
+    }
+
+    static {
+      defaultInstance = new StatCollectRequest(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:StatCollectRequest)
+  }
+
+  public interface StatCollectResponseOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // required uint64 rowsScanned = 1;
+    /**
+     * <code>required uint64 rowsScanned = 1;</code>
+     */
+    boolean hasRowsScanned();
+    /**
+     * <code>required uint64 rowsScanned = 1;</code>
+     */
+    long getRowsScanned();
+  }
+  /**
+   * Protobuf type {@code StatCollectResponse}
+   */
+  public static final class StatCollectResponse extends
+      com.google.protobuf.GeneratedMessage
+      implements StatCollectResponseOrBuilder {
+    // Use StatCollectResponse.newBuilder() to construct.
+    private StatCollectResponse(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private StatCollectResponse(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final StatCollectResponse defaultInstance;
+    public static StatCollectResponse getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public StatCollectResponse getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private StatCollectResponse(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 8: {
+              bitField0_ |= 0x00000001;
+              rowsScanned_ = input.readUInt64();
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.phoenix.coprocessor.generated.StatCollectorProtos.internal_static_StatCollectResponse_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.phoenix.coprocessor.generated.StatCollectorProtos.internal_static_StatCollectResponse_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse.class, org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<StatCollectResponse> PARSER =
+        new com.google.protobuf.AbstractParser<StatCollectResponse>() {
+      public StatCollectResponse parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new StatCollectResponse(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<StatCollectResponse> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // required uint64 rowsScanned = 1;
+    public static final int ROWSSCANNED_FIELD_NUMBER = 1;
+    private long rowsScanned_;
+    /**
+     * <code>required uint64 rowsScanned = 1;</code>
+     */
+    public boolean hasRowsScanned() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required uint64 rowsScanned = 1;</code>
+     */
+    public long getRowsScanned() {
+      return rowsScanned_;
+    }
+
+    private void initFields() {
+      rowsScanned_ = 0L;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (!hasRowsScanned()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeUInt64(1, rowsScanned_);
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeUInt64Size(1, rowsScanned_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse)) {
+        return super.equals(obj);
+      }
+      org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse other = (org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse) obj;
+
+      boolean result = true;
+      result = result && (hasRowsScanned() == other.hasRowsScanned());
+      if (hasRowsScanned()) {
+        result = result && (getRowsScanned()
+            == other.getRowsScanned());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasRowsScanned()) {
+        hash = (37 * hash) + ROWSSCANNED_FIELD_NUMBER;
+        hash = (53 * hash) + hashLong(getRowsScanned());
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code StatCollectResponse}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponseOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.phoenix.coprocessor.generated.StatCollectorProtos.internal_static_StatCollectResponse_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.phoenix.coprocessor.generated.StatCollectorProtos.internal_static_StatCollectResponse_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse.class, org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse.Builder.class);
+      }
+
+      // Construct using org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        rowsScanned_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.phoenix.coprocessor.generated.StatCollectorProtos.internal_static_StatCollectResponse_descriptor;
+      }
+
+      public org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse getDefaultInstanceForType() {
+        return org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse.getDefaultInstance();
+      }
+
+      public org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse build() {
+        org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse buildPartial() {
+        org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse result = new org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.rowsScanned_ = rowsScanned_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse) {
+          return mergeFrom((org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse other) {
+        if (other == org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse.getDefaultInstance()) return this;
+        if (other.hasRowsScanned()) {
+          setRowsScanned(other.getRowsScanned());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasRowsScanned()) {
+          
+          return false;
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // required uint64 rowsScanned = 1;
+      private long rowsScanned_ ;
+      /**
+       * <code>required uint64 rowsScanned = 1;</code>
+       */
+      public boolean hasRowsScanned() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required uint64 rowsScanned = 1;</code>
+       */
+      public long getRowsScanned() {
+        return rowsScanned_;
+      }
+      /**
+       * <code>required uint64 rowsScanned = 1;</code>
+       */
+      public Builder setRowsScanned(long value) {
+        bitField0_ |= 0x00000001;
+        rowsScanned_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required uint64 rowsScanned = 1;</code>
+       */
+      public Builder clearRowsScanned() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        rowsScanned_ = 0L;
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:StatCollectResponse)
+    }
+
+    static {
+      defaultInstance = new StatCollectResponse(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:StatCollectResponse)
+  }
+
+  /**
+   * Protobuf service {@code StatCollectService}
+   */
+  public static abstract class StatCollectService
+      implements com.google.protobuf.Service {
+    protected StatCollectService() {}
+
+    public interface Interface {
+      /**
+       * <code>rpc collectStat(.StatCollectRequest) returns (.StatCollectResponse);</code>
+       */
+      public abstract void collectStat(
+          com.google.protobuf.RpcController controller,
+          org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest request,
+          com.google.protobuf.RpcCallback<org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse> done);
+
+    }
+
+    public static com.google.protobuf.Service newReflectiveService(
+        final Interface impl) {
+      return new StatCollectService() {
+        @java.lang.Override
+        public  void collectStat(
+            com.google.protobuf.RpcController controller,
+            org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest request,
+            com.google.protobuf.RpcCallback<org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse> done) {
+          impl.collectStat(controller, request, done);
+        }
+
+      };
+    }
+
+    public static com.google.protobuf.BlockingService
+        newReflectiveBlockingService(final BlockingInterface impl) {
+      return new com.google.protobuf.BlockingService() {
+        public final com.google.protobuf.Descriptors.ServiceDescriptor
+            getDescriptorForType() {
+          return getDescriptor();
+        }
+
+        public final com.google.protobuf.Message callBlockingMethod(
+            com.google.protobuf.Descriptors.MethodDescriptor method,
+            com.google.protobuf.RpcController controller,
+            com.google.protobuf.Message request)
+            throws com.google.protobuf.ServiceException {
+          if (method.getService() != getDescriptor()) {
+            throw new java.lang.IllegalArgumentException(
+              "Service.callBlockingMethod() given method descriptor for " +
+              "wrong service type.");
+          }
+          switch(method.getIndex()) {
+            case 0:
+              return impl.collectStat(controller, (org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest)request);
+            default:
+              throw new java.lang.AssertionError("Can't get here.");
+          }
+        }
+
+        public final com.google.protobuf.Message
+            getRequestPrototype(
+            com.google.protobuf.Descriptors.MethodDescriptor method) {
+          if (method.getService() != getDescriptor()) {
+            throw new java.lang.IllegalArgumentException(
+              "Service.getRequestPrototype() given method " +
+              "descriptor for wrong service type.");
+          }
+          switch(method.getIndex()) {
+            case 0:
+              return org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest.getDefaultInstance();
+            default:
+              throw new java.lang.AssertionError("Can't get here.");
+          }
+        }
+
+        public final com.google.protobuf.Message
+            getResponsePrototype(
+            com.google.protobuf.Descriptors.MethodDescriptor method) {
+          if (method.getService() != getDescriptor()) {
+            throw new java.lang.IllegalArgumentException(
+              "Service.getResponsePrototype() given method " +
+              "descriptor for wrong service type.");
+          }
+          switch(method.getIndex()) {
+            case 0:
+              return org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse.getDefaultInstance();
+            default:
+              throw new java.lang.AssertionError("Can't get here.");
+          }
+        }
+
+      };
+    }
+
+    /**
+     * <code>rpc collectStat(.StatCollectRequest) returns (.StatCollectResponse);</code>
+     */
+    public abstract void collectStat(
+        com.google.protobuf.RpcController controller,
+        org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest request,
+        com.google.protobuf.RpcCallback<org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse> done);
+
+    public static final
+        com.google.protobuf.Descriptors.ServiceDescriptor
+        getDescriptor() {
+      return org.apache.phoenix.coprocessor.generated.StatCollectorProtos.getDescriptor().getServices().get(0);
+    }
+    public final com.google.protobuf.Descriptors.ServiceDescriptor
+        getDescriptorForType() {
+      return getDescriptor();
+    }
+
+    public final void callMethod(
+        com.google.protobuf.Descriptors.MethodDescriptor method,
+        com.google.protobuf.RpcController controller,
+        com.google.protobuf.Message request,
+        com.google.protobuf.RpcCallback<
+          com.google.protobuf.Message> done) {
+      if (method.getService() != getDescriptor()) {
+        throw new java.lang.IllegalArgumentException(
+          "Service.callMethod() given method descriptor for wrong " +
+          "service type.");
+      }
+      switch(method.getIndex()) {
+        case 0:
+          this.collectStat(controller, (org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest)request,
+            com.google.protobuf.RpcUtil.<org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse>specializeCallback(
+              done));
+          return;
+        default:
+          throw new java.lang.AssertionError("Can't get here.");
+      }
+    }
+
+    public final com.google.protobuf.Message
+        getRequestPrototype(
+        com.google.protobuf.Descriptors.MethodDescriptor method) {
+      if (method.getService() != getDescriptor()) {
+        throw new java.lang.IllegalArgumentException(
+          "Service.getRequestPrototype() given method " +
+          "descriptor for wrong service type.");
+      }
+      switch(method.getIndex()) {
+        case 0:
+          return org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest.getDefaultInstance();
+        default:
+          throw new java.lang.AssertionError("Can't get here.");
+      }
+    }
+
+    public final com.google.protobuf.Message
+        getResponsePrototype(
+        com.google.protobuf.Descriptors.MethodDescriptor method) {
+      if (method.getService() != getDescriptor()) {
+        throw new java.lang.IllegalArgumentException(
+          "Service.getResponsePrototype() given method " +
+          "descriptor for wrong service type.");
+      }
+      switch(method.getIndex()) {
+        case 0:
+          return org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse.getDefaultInstance();
+        default:
+          throw new java.lang.AssertionError("Can't get here.");
+      }
+    }
+
+    public static Stub newStub(
+        com.google.protobuf.RpcChannel channel) {
+      return new Stub(channel);
+    }
+
+    public static final class Stub extends org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectService implements Interface {
+      private Stub(com.google.protobuf.RpcChannel channel) {
+        this.channel = channel;
+      }
+
+      private final com.google.protobuf.RpcChannel channel;
+
+      public com.google.protobuf.RpcChannel getChannel() {
+        return channel;
+      }
+
+      public  void collectStat(
+          com.google.protobuf.RpcController controller,
+          org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest request,
+          com.google.protobuf.RpcCallback<org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(0),
+          controller,
+          request,
+          org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse.class,
+            org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse.getDefaultInstance()));
+      }
+    }
+
+    public static BlockingInterface newBlockingStub(
+        com.google.protobuf.BlockingRpcChannel channel) {
+      return new BlockingStub(channel);
+    }
+
+    public interface BlockingInterface {
+      public org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse collectStat(
+          com.google.protobuf.RpcController controller,
+          org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest request)
+          throws com.google.protobuf.ServiceException;
+    }
+
+    private static final class BlockingStub implements BlockingInterface {
+      private BlockingStub(com.google.protobuf.BlockingRpcChannel channel) {
+        this.channel = channel;
+      }
+
+      private final com.google.protobuf.BlockingRpcChannel channel;
+
+      public org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse collectStat(
+          com.google.protobuf.RpcController controller,
+          org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectRequest request)
+          throws com.google.protobuf.ServiceException {
+        return (org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse) channel.callBlockingMethod(
+          getDescriptor().getMethods().get(0),
+          controller,
+          request,
+          org.apache.phoenix.coprocessor.generated.StatCollectorProtos.StatCollectResponse.getDefaultInstance());
+      }
+
+    }
+
+    // @@protoc_insertion_point(class_scope:StatCollectService)
+  }
+
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_StatCollectRequest_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_StatCollectRequest_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_StatCollectResponse_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_StatCollectResponse_fieldAccessorTable;
+
+  public static com.google.protobuf.Descriptors.FileDescriptor
+      getDescriptor() {
+    return descriptor;
+  }
+  private static com.google.protobuf.Descriptors.FileDescriptor
+      descriptor;
+  static {
+    java.lang.String[] descriptorData = {
+      "\n\027StatisticsCollect.proto\"7\n\022StatCollect" +
+      "Request\022\020\n\010startRow\030\001 \001(\014\022\017\n\007stopRow\030\002 \001" +
+      "(\014\"*\n\023StatCollectResponse\022\023\n\013rowsScanned" +
+      "\030\001 \002(\0042N\n\022StatCollectService\0228\n\013collectS" +
+      "tat\022\023.StatCollectRequest\032\024.StatCollectRe" +
+      "sponseBG\n(org.apache.phoenix.coprocessor" +
+      ".generatedB\023StatCollectorProtosH\001\210\001\001\240\001\001"
+    };
+    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+      new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
+        public com.google.protobuf.ExtensionRegistry assignDescriptors(
+            com.google.protobuf.Descriptors.FileDescriptor root) {
+          descriptor = root;
+          internal_static_StatCollectRequest_descriptor =
+            getDescriptor().getMessageTypes().get(0);
+          internal_static_StatCollectRequest_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_StatCollectRequest_descriptor,
+              new java.lang.String[] { "StartRow", "StopRow", });
+          internal_static_StatCollectResponse_descriptor =
+            getDescriptor().getMessageTypes().get(1);
+          internal_static_StatCollectResponse_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_StatCollectResponse_descriptor,
+              new java.lang.String[] { "RowsScanned", });
+          return null;
+        }
+      };
+    com.google.protobuf.Descriptors.FileDescriptor
+      .internalBuildGeneratedFileFrom(descriptorData,
+        new com.google.protobuf.Descriptors.FileDescriptor[] {
+        }, assigner);
+  }
+
+  // @@protoc_insertion_point(outer_class_scope)
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/main/java/org/apache/phoenix/iterate/DefaultParallelIteratorRegionSplitter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/DefaultParallelIteratorRegionSplitter.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/DefaultParallelIteratorRegionSplitter.java
index 1498480..2f568db 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/iterate/DefaultParallelIteratorRegionSplitter.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/DefaultParallelIteratorRegionSplitter.java
@@ -18,33 +18,28 @@
 package org.apache.phoenix.iterate;
 
 import java.sql.SQLException;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 
-import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.util.Bytes;
-
-import com.google.common.base.Preconditions;
-import com.google.common.base.Predicate;
-import com.google.common.collect.ArrayListMultimap;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.ListMultimap;
-import com.google.common.collect.Lists;
 import org.apache.phoenix.compile.StatementContext;
 import org.apache.phoenix.parse.HintNode;
-import org.apache.phoenix.parse.HintNode.Hint;
 import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
-import org.apache.phoenix.query.StatsManager;
+import org.apache.phoenix.schema.ColumnFamilyNotFoundException;
 import org.apache.phoenix.schema.PTable;
-import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.TableRef;
-import org.apache.phoenix.schema.PTable.IndexType;
 import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.SchemaUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
 
 
 /**
@@ -56,12 +51,11 @@ import org.apache.phoenix.util.ReadOnlyProps;
  */
 public class DefaultParallelIteratorRegionSplitter implements ParallelIteratorRegionSplitter {
 
-    protected final int targetConcurrency;
-    protected final int maxConcurrency;
-    protected final int maxIntraRegionParallelization;
+    protected final long guidePostsDepth;
     protected final StatementContext context;
     protected final TableRef tableRef;
 
+    private static final Logger logger = LoggerFactory.getLogger(DefaultParallelIteratorRegionSplitter.class);
     public static DefaultParallelIteratorRegionSplitter getInstance(StatementContext context, TableRef table, HintNode hintNode) {
         return new DefaultParallelIteratorRegionSplitter(context, table, hintNode);
     }
@@ -70,22 +64,16 @@ public class DefaultParallelIteratorRegionSplitter implements ParallelIteratorRe
         this.context = context;
         this.tableRef = table;
         ReadOnlyProps props = context.getConnection().getQueryServices().getProps();
-        this.targetConcurrency = props.getInt(QueryServices.TARGET_QUERY_CONCURRENCY_ATTRIB,
-                QueryServicesOptions.DEFAULT_TARGET_QUERY_CONCURRENCY);
-        this.maxConcurrency = props.getInt(QueryServices.MAX_QUERY_CONCURRENCY_ATTRIB,
-                QueryServicesOptions.DEFAULT_MAX_QUERY_CONCURRENCY);
-        Preconditions.checkArgument(targetConcurrency >= 1, "Invalid target concurrency: " + targetConcurrency);
-        Preconditions.checkArgument(maxConcurrency >= targetConcurrency , "Invalid max concurrency: " + maxConcurrency);
-        this.maxIntraRegionParallelization = hintNode.hasHint(Hint.NO_INTRA_REGION_PARALLELIZATION) ? 1 : props.getInt(QueryServices.MAX_INTRA_REGION_PARALLELIZATION_ATTRIB,
-                QueryServicesOptions.DEFAULT_MAX_INTRA_REGION_PARALLELIZATION);
-        Preconditions.checkArgument(maxIntraRegionParallelization >= 1 , "Invalid max intra region parallelization: " + maxIntraRegionParallelization);
+        this.guidePostsDepth = props.getLong(QueryServices.HISTOGRAM_BYTE_DEPTH_CONF_KEY,
+                QueryServicesOptions.DEFAULT_HISTOGRAM_BYTE_DEPTH);
     }
 
     // Get the mapping between key range and the regions that contains them.
     protected List<HRegionLocation> getAllRegions() throws SQLException {
         Scan scan = context.getScan();
         PTable table = tableRef.getTable();
-        List<HRegionLocation> allTableRegions = context.getConnection().getQueryServices().getAllTableRegions(table.getPhysicalName().getBytes());
+        List<HRegionLocation> allTableRegions = context.getConnection().getQueryServices()
+                .getAllTableRegions(table.getPhysicalName().getBytes());
         // If we're not salting, then we've already intersected the minMaxRange with the scan range
         // so there's nothing to do here.
         return filterRegions(allTableRegions, scan.getStartRow(), scan.getStopRow());
@@ -109,7 +97,8 @@ public class DefaultParallelIteratorRegionSplitter implements ParallelIteratorRe
         regions = Iterables.filter(allTableRegions, new Predicate<HRegionLocation>() {
             @Override
             public boolean apply(HRegionLocation location) {
-                KeyRange regionKeyRange = KeyRange.getKeyRange(location.getRegionInfo().getStartKey(), location.getRegionInfo().getEndKey());
+                KeyRange regionKeyRange = KeyRange.getKeyRange(location.getRegionInfo().getStartKey(), location
+                        .getRegionInfo().getEndKey());
                 return keyRange.intersect(regionKeyRange) != KeyRange.EMPTY_RANGE;
             }
         });
@@ -117,121 +106,68 @@ public class DefaultParallelIteratorRegionSplitter implements ParallelIteratorRe
     }
 
     protected List<KeyRange> genKeyRanges(List<HRegionLocation> regions) {
-        if (regions.isEmpty()) {
-            return Collections.emptyList();
-        }
-        
-        StatsManager statsManager = context.getConnection().getQueryServices().getStatsManager();
-        // the splits are computed as follows:
-        //
-        // let's suppose:
-        // t = target concurrency
-        // m = max concurrency
-        // r = the number of regions we need to scan
-        //
-        // if r >= t:
-        //    scan using regional boundaries
-        // elif r > t/2:
-        //    split each region in s splits such that:
-        //    s = max(x) where s * x < m
-        // else:
-        //    split each region in s splits such that:
-        //    s = max(x) where s * x < t
-        //
-        // The idea is to align splits with region boundaries. If rows are not evenly
-        // distributed across regions, using this scheme compensates for regions that
-        // have more rows than others, by applying tighter splits and therefore spawning
-        // off more scans over the overloaded regions.
-        int splitsPerRegion = getSplitsPerRegion(regions.size());
-        // Create a multi-map of ServerName to List<KeyRange> which we'll use to round robin from to ensure
-        // that we keep each region server busy for each query.
-        ListMultimap<HRegionLocation,KeyRange> keyRangesPerRegion = ArrayListMultimap.create(regions.size(),regions.size() * splitsPerRegion);;
-        if (splitsPerRegion == 1) {
-            for (HRegionLocation region : regions) {
-                keyRangesPerRegion.put(region, ParallelIterators.TO_KEY_RANGE.apply(region));
-            }
+        if (regions.isEmpty()) { return Collections.emptyList(); }
+        Scan scan = context.getScan();
+        PTable table = this.tableRef.getTable();
+        byte[] defaultCF = SchemaUtil.getEmptyColumnFamily(table);
+        List<byte[]> gps = Lists.newArrayList();
+
+        if (table.getColumnFamilies().isEmpty()) {
+            // For sure we can get the defaultCF from the table
+            gps = table.getGuidePosts();
         } else {
-            // Maintain bucket for each server and then returns KeyRanges in round-robin
-            // order to ensure all servers are utilized.
-            for (HRegionLocation region : regions) {
-                byte[] startKey = region.getRegionInfo().getStartKey();
-                byte[] stopKey = region.getRegionInfo().getEndKey();
-                boolean lowerUnbound = Bytes.compareTo(startKey, HConstants.EMPTY_START_ROW) == 0;
-                boolean upperUnbound = Bytes.compareTo(stopKey, HConstants.EMPTY_END_ROW) == 0;
-                /*
-                 * If lower/upper unbound, get the min/max key from the stats manager.
-                 * We use this as the boundary to split on, but we still use the empty
-                 * byte as the boundary in the actual scan (in case our stats are out
-                 * of date).
-                 */
-                if (lowerUnbound) {
-                    startKey = statsManager.getMinKey(tableRef);
-                    if (startKey == null) {
-                        keyRangesPerRegion.put(region,ParallelIterators.TO_KEY_RANGE.apply(region));
-                        continue;
+            try {
+                if (scan.getFamilyMap().size() > 0) {
+                    if (scan.getFamilyMap().containsKey(defaultCF)) { // Favor using default CF if it's used in scan
+                        gps = table.getColumnFamily(defaultCF).getGuidePosts();
+                    } else { // Otherwise, just use first CF in use by scan
+                        gps = table.getColumnFamily(scan.getFamilyMap().keySet().iterator().next()).getGuidePosts();
                     }
-                }
-                if (upperUnbound) {
-                    stopKey = statsManager.getMaxKey(tableRef);
-                    if (stopKey == null) {
-                        keyRangesPerRegion.put(region,ParallelIterators.TO_KEY_RANGE.apply(region));
-                        continue;
-                    }
-                }
-                
-                byte[][] boundaries = null;
-                // Both startKey and stopKey will be empty the first time
-                if (Bytes.compareTo(startKey, stopKey) >= 0 || (boundaries = Bytes.split(startKey, stopKey, splitsPerRegion - 1)) == null) {
-                    // Bytes.split may return null if the key space
-                    // between start and end key is too small
-                    keyRangesPerRegion.put(region,ParallelIterators.TO_KEY_RANGE.apply(region));
                 } else {
-                    keyRangesPerRegion.put(region,KeyRange.getKeyRange(lowerUnbound ? KeyRange.UNBOUND : boundaries[0], boundaries[1]));
-                    if (boundaries.length > 1) {
-                        for (int i = 1; i < boundaries.length-2; i++) {
-                            keyRangesPerRegion.put(region,KeyRange.getKeyRange(boundaries[i], true, boundaries[i+1], false));
-                        }
-                        keyRangesPerRegion.put(region,KeyRange.getKeyRange(boundaries[boundaries.length-2], true, upperUnbound ? KeyRange.UNBOUND : boundaries[boundaries.length-1], false));
-                    }
+                    gps = table.getColumnFamily(defaultCF).getGuidePosts();
                 }
+            } catch (ColumnFamilyNotFoundException cfne) {
+                // Alter table does this
             }
         }
-        List<KeyRange> splits = Lists.newArrayListWithCapacity(regions.size() * splitsPerRegion);
-        // as documented for ListMultimap
-        Collection<Collection<KeyRange>> values = keyRangesPerRegion.asMap().values();
-        List<Collection<KeyRange>> keyRangesList = Lists.newArrayList(values);
-        // Randomize range order to ensure that we don't hit the region servers in the same order every time
-        // thus helping to distribute the load more evenly
-        Collections.shuffle(keyRangesList);
-        // Transpose values in map to get regions in round-robin server order. This ensures that
-        // all servers will be used to process the set of parallel threads available in our executor.
-        int i = 0;
-        boolean done;
-        do {
-            done = true;
-            for (int j = 0; j < keyRangesList.size(); j++) {
-                List<KeyRange> keyRanges = (List<KeyRange>)keyRangesList.get(j);
-                if (i < keyRanges.size()) {
-                    splits.add(keyRanges.get(i));
-                    done = false;
+        List<KeyRange> guidePosts = Lists.newArrayListWithCapacity(regions.size());
+        byte[] currentKey = regions.get(0).getRegionInfo().getStartKey();
+        byte[] endKey = null;
+        int regionIndex = 0;
+        int guideIndex = 0;
+        int gpsSize = gps.size();
+        int regionSize = regions.size();
+        if (currentKey.length > 0) {
+            guideIndex = Collections.binarySearch(gps, currentKey, Bytes.BYTES_COMPARATOR);
+            guideIndex = (guideIndex < 0 ? -(guideIndex + 1) : (guideIndex + 1));
+        }
+        // Merge bisect with guideposts for all but the last region
+        while (regionIndex < regionSize) {
+            byte[] currentGuidePost;
+            currentKey = regions.get(regionIndex).getRegionInfo().getStartKey();
+            endKey = regions.get(regionIndex++).getRegionInfo().getEndKey();
+            while (guideIndex < gpsSize
+                    && (Bytes.compareTo(currentGuidePost = gps.get(guideIndex), endKey) <= 0 || endKey.length == 0)) {
+                KeyRange keyRange = KeyRange.getKeyRange(currentKey, currentGuidePost);
+                if (keyRange != KeyRange.EMPTY_RANGE) {
+                    guidePosts.add(keyRange);
                 }
+                currentKey = currentGuidePost;
+                guideIndex++;
+            }
+            KeyRange keyRange = KeyRange.getKeyRange(currentKey, endKey);
+            if (keyRange != KeyRange.EMPTY_RANGE) {
+                guidePosts.add(keyRange);
             }
-            i++;
-        } while (!done);
-        return splits;
+        }
+        if (logger.isDebugEnabled()) {
+            logger.debug("The captured guideposts are: " + guidePosts);
+        }
+        return guidePosts;
     }
-
+        
     @Override
     public List<KeyRange> getSplits() throws SQLException {
         return genKeyRanges(getAllRegions());
     }
-
-    @Override
-    public int getSplitsPerRegion(int numRegions) {
-        int splitsPerRegion =
-                numRegions >= targetConcurrency ? 1
-                        : (numRegions > targetConcurrency / 2 ? maxConcurrency : targetConcurrency)
-                                / numRegions;
-        return Math.min(splitsPerRegion, maxIntraRegionParallelization);
-    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/main/java/org/apache/phoenix/iterate/LocalIndexParallelIteratorRegionSplitter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/LocalIndexParallelIteratorRegionSplitter.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/LocalIndexParallelIteratorRegionSplitter.java
index c3a38d5..14da71e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/iterate/LocalIndexParallelIteratorRegionSplitter.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/LocalIndexParallelIteratorRegionSplitter.java
@@ -39,9 +39,5 @@ public class LocalIndexParallelIteratorRegionSplitter extends DefaultParallelIte
     protected List<HRegionLocation> getAllRegions() throws SQLException {
         return context.getConnection().getQueryServices().getAllTableRegions(tableRef.getTable().getPhysicalName().getBytes());
     }
-
-    @Override
-    public int getSplitsPerRegion(int numRegions) {
-        return 1;
-    }
+    
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/main/java/org/apache/phoenix/iterate/ParallelIteratorRegionSplitter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/ParallelIteratorRegionSplitter.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/ParallelIteratorRegionSplitter.java
index efd9eec..74029fd 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/iterate/ParallelIteratorRegionSplitter.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/ParallelIteratorRegionSplitter.java
@@ -32,6 +32,4 @@ public interface ParallelIteratorRegionSplitter {
 
     public List<KeyRange> getSplits() throws SQLException;
 
-    public int getSplitsPerRegion(int numRegions);
-
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
index 99b5910..a800fd9 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
@@ -106,6 +106,10 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData, org.apache.pho
     public static final byte[] SYSTEM_CATALOG_TABLE_BYTES = Bytes.toBytes(SYSTEM_CATALOG_SCHEMA);
     public static final String SYSTEM_CATALOG_NAME = SchemaUtil.getTableName(SYSTEM_CATALOG_SCHEMA, SYSTEM_CATALOG_TABLE);
     public static final byte[] SYSTEM_CATALOG_NAME_BYTES = SchemaUtil.getTableNameAsBytes(SYSTEM_CATALOG_TABLE_BYTES, SYSTEM_CATALOG_SCHEMA_BYTES);
+    public static final String SYSTEM_STATS_TABLE = "STATS";
+    public static final byte[] SYSTEM_STATS_BYTES = Bytes.toBytes(SYSTEM_STATS_TABLE);
+    public static final String SYSTEM_STATS_NAME = SchemaUtil.getTableName(SYSTEM_CATALOG_SCHEMA, SYSTEM_STATS_TABLE);
+    public static final byte[] SYSTEM_STATS_NAME_BYTES = SchemaUtil.getTableNameAsBytes(SYSTEM_CATALOG_TABLE_BYTES, SYSTEM_STATS_BYTES);
     
     public static final String SYSTEM_CATALOG_ALIAS = "\"SYSTEM.TABLE\"";
 
@@ -113,6 +117,8 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData, org.apache.pho
     public static final byte[] TABLE_NAME_BYTES = Bytes.toBytes(TABLE_NAME);
     public static final String TABLE_TYPE = "TABLE_TYPE";
     public static final byte[] TABLE_TYPE_BYTES = Bytes.toBytes(TABLE_TYPE);
+    public static final String PHYSICAL_NAME = "PHYSICAL_NAME";
+    public static final byte[] PHYSICAL_NAME_BYTES = Bytes.toBytes(PHYSICAL_NAME);
     
     public static final String COLUMN_FAMILY = "COLUMN_FAMILY";
     public static final String TABLE_CAT = "TABLE_CAT";
@@ -224,6 +230,16 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData, org.apache.pho
     public static final String INDEX_DISABLE_TIMESTAMP = "INDEX_DISABLE_TIMESTAMP";
     public static final byte[] INDEX_DISABLE_TIMESTAMP_BYTES = Bytes.toBytes(INDEX_DISABLE_TIMESTAMP);
     
+    public static final String REGION_NAME = "REGION_NAME";
+    public static final byte[] REGION_NAME_BYTES = Bytes.toBytes(REGION_NAME);
+    public static final String GUIDE_POSTS = "GUIDE_POSTS";
+    public static final byte[] GUIDE_POSTS_BYTES = Bytes.toBytes(GUIDE_POSTS);
+    public static final String MIN_KEY = "MIN_KEY";
+    public static final byte[] MIN_KEY_BYTES = Bytes.toBytes(MIN_KEY);
+    public static final String MAX_KEY = "MAX_KEY";
+    public static final byte[] MAX_KEY_BYTES = Bytes.toBytes(MAX_KEY);
+    public static final String LAST_STATS_UPDATE_TIME = "LAST_STATS_UPDATE_TIME";
+    public static final byte[] LAST_STATS_UPDATE_TIME_BYTES = Bytes.toBytes(LAST_STATS_UPDATE_TIME);
     private final PhoenixConnection connection;
     private final ResultSet emptyResultSet;
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
index 00d271d..f7d6e14 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
@@ -91,6 +91,7 @@ import org.apache.phoenix.parse.SQLParser;
 import org.apache.phoenix.parse.SelectStatement;
 import org.apache.phoenix.parse.TableName;
 import org.apache.phoenix.parse.TableNode;
+import org.apache.phoenix.parse.UpdateStatisticsStatement;
 import org.apache.phoenix.parse.UpsertStatement;
 import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.query.QueryConstants;
@@ -650,6 +651,49 @@ public class PhoenixStatement implements Statement, SQLCloseable, org.apache.pho
             };
         }
     }
+    
+    private static class ExecutableUpdateStatisticsStatement extends UpdateStatisticsStatement implements
+            CompilableStatement {
+
+        public ExecutableUpdateStatisticsStatement(NamedTableNode table) {
+            super(table);
+        }
+
+        @SuppressWarnings("unchecked")
+        @Override
+        public MutationPlan compilePlan(final PhoenixStatement stmt, Sequence.ValueOp seqAction) throws SQLException {
+            final StatementContext context = new StatementContext(stmt);
+            return new MutationPlan() {
+
+                @Override
+                public StatementContext getContext() {
+                    return context;
+                }
+
+                @Override
+                public ParameterMetaData getParameterMetaData() {
+                    return PhoenixParameterMetaData.EMPTY_PARAMETER_META_DATA;
+                }
+
+                @Override
+                public ExplainPlan getExplainPlan() throws SQLException {
+                    return new ExplainPlan(Collections.singletonList("ANALYZE"));
+                }
+
+                @Override
+                public PhoenixConnection getConnection() {
+                    return stmt.getConnection();
+                }
+
+                @Override
+                public MutationState execute() throws SQLException {
+                    MetaDataClient client = new MetaDataClient(getConnection());
+                    return client.updateStatistics(ExecutableUpdateStatisticsStatement.this);
+                }
+            };
+        }
+
+    }
 
     private static class ExecutableAddColumnStatement extends AddColumnStatement implements CompilableStatement {
 
@@ -806,6 +850,11 @@ public class PhoenixStatement implements Statement, SQLCloseable, org.apache.pho
         public ExplainStatement explain(BindableStatement statement) {
             return new ExecutableExplainStatement(statement);
         }
+
+        @Override
+        public UpdateStatisticsStatement updateStatistics(NamedTableNode table) {
+            return new ExecutableUpdateStatisticsStatement(table);
+        }
     }
     
     static class PhoenixStatementParser extends SQLParser {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
index e16849f..16423e0 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
@@ -339,6 +339,10 @@ public class ParseNodeFactory {
     public DivideParseNode divide(List<ParseNode> children) {
         return new DivideParseNode(children);
     }
+    
+    public UpdateStatisticsStatement updateStatistics(NamedTableNode table) {
+      return new UpdateStatisticsStatement(table);
+    }
 
 
     public FunctionParseNode functionDistinct(String name, List<ParseNode> args) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/main/java/org/apache/phoenix/parse/UpdateStatisticsStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/UpdateStatisticsStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/UpdateStatisticsStatement.java
new file mode 100644
index 0000000..9eff74a
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/UpdateStatisticsStatement.java
@@ -0,0 +1,26 @@
+/*
+ * 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.parse;
+
+public class UpdateStatisticsStatement extends SingleTableStatement {
+
+    public UpdateStatisticsStatement(NamedTableNode table) {
+        super(table, 0);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java
index ddf6b61..0c1f45d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java
@@ -63,8 +63,6 @@ public interface ConnectionQueryServices extends QueryServices, MetaDataMutated
 
     public HTableDescriptor getTableDescriptor(byte[] tableName) throws SQLException;
 
-    public StatsManager getStatsManager();
-
     public List<HRegionLocation> getAllTableRegions(byte[] tableName) throws SQLException;
 
     public PhoenixConnection connect(String url, Properties info) throws SQLException;
@@ -96,6 +94,9 @@ public interface ConnectionQueryServices extends QueryServices, MetaDataMutated
     void addConnection(PhoenixConnection connection) throws SQLException;
     void removeConnection(PhoenixConnection connection) throws SQLException;
 
+    long updateStatistics(KeyRange keyRange, byte[] tableName)
+            throws SQLException;
+
     /**
      * @return the {@link KeyValueBuilder} that is valid for the locally installed version of HBase.
      */
@@ -105,4 +106,5 @@ public interface ConnectionQueryServices extends QueryServices, MetaDataMutated
     public boolean supportsFeature(Feature feature);
     
     public String getUserName();
+    public void clearCacheForTable(final byte[] tenantId, final byte[] schemaName, final byte[] tableName, long clientTS) throws SQLException;
 }
\ No newline at end of file


[5/5] git commit: PHOENIX-180 Use stats to guide query parallelization (Ramkrishna S Vasudevan)

Posted by ja...@apache.org.
PHOENIX-180 Use stats to guide query parallelization (Ramkrishna S Vasudevan)

Conflicts:
	phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java


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

Branch: refs/heads/4.0
Commit: 5cdc938e8f6ffc7db629f39951270e89dd4873b1
Parents: a18862d
Author: James Taylor <jt...@salesforce.com>
Authored: Fri Sep 19 16:38:28 2014 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Fri Sep 19 16:48:14 2014 -0700

----------------------------------------------------------------------
 .../org/apache/phoenix/end2end/ArrayIT.java     |   12 +-
 .../end2end/BaseClientManagedTimeIT.java        |    2 +-
 .../BaseParallelIteratorsRegionSplitterIT.java  |    9 +-
 .../org/apache/phoenix/end2end/BaseQueryIT.java |    8 +-
 .../org/apache/phoenix/end2end/BaseViewIT.java  |    2 -
 ...efaultParallelIteratorsRegionSplitterIT.java |   55 +-
 .../phoenix/end2end/GuidePostsLifeCycleIT.java  |  148 ++
 .../org/apache/phoenix/end2end/HashJoinIT.java  |    2 -
 .../org/apache/phoenix/end2end/KeyOnlyIT.java   |   61 +-
 .../phoenix/end2end/MultiCfQueryExecIT.java     |   78 +-
 .../end2end/QueryDatabaseMetaDataIT.java        |    4 +
 .../org/apache/phoenix/end2end/QueryIT.java     |   39 +-
 .../apache/phoenix/end2end/ReverseScanIT.java   |    2 -
 ...ipRangeParallelIteratorRegionSplitterIT.java |   32 +-
 .../phoenix/end2end/StatsCollectorIT.java       |  231 ++++
 .../apache/phoenix/end2end/StatsManagerIT.java  |    2 +-
 .../end2end/TenantSpecificTablesDDLIT.java      |    4 +
 .../end2end/TenantSpecificTablesDMLIT.java      |   23 +-
 .../end2end/TenantSpecificViewIndexIT.java      |    2 +-
 .../phoenix/end2end/index/MutableIndexIT.java   |    1 -
 .../phoenix/end2end/index/SaltedIndexIT.java    |   17 +-
 .../salted/SaltedTableUpsertSelectIT.java       |   21 +-
 phoenix-core/src/main/antlr3/PhoenixSQL.g       |   10 +-
 .../org/apache/phoenix/cache/GlobalCache.java   |    7 +-
 .../coprocessor/MetaDataEndpointImpl.java       |  132 +-
 .../coprocessor/generated/MetaDataProtos.java   | 1223 ++++++++++++++++-
 .../generated/StatCollectorProtos.java          | 1269 ++++++++++++++++++
 .../DefaultParallelIteratorRegionSplitter.java  |  198 +--
 ...ocalIndexParallelIteratorRegionSplitter.java |    6 +-
 .../iterate/ParallelIteratorRegionSplitter.java |    2 -
 .../phoenix/jdbc/PhoenixDatabaseMetaData.java   |   16 +
 .../apache/phoenix/jdbc/PhoenixStatement.java   |   49 +
 .../apache/phoenix/parse/ParseNodeFactory.java  |    4 +
 .../parse/UpdateStatisticsStatement.java        |   26 +
 .../phoenix/query/ConnectionQueryServices.java  |    6 +-
 .../query/ConnectionQueryServicesImpl.java      |  164 ++-
 .../query/ConnectionlessQueryServicesImpl.java  |   43 +-
 .../query/DelegateConnectionQueryServices.java  |   16 +-
 .../apache/phoenix/query/QueryConstants.java    |   28 +-
 .../org/apache/phoenix/query/QueryServices.java |    8 +-
 .../phoenix/query/QueryServicesOptions.java     |   34 +-
 .../apache/phoenix/schema/MetaDataClient.java   |  225 +++-
 .../apache/phoenix/schema/PColumnFamily.java    |    3 +
 .../phoenix/schema/PColumnFamilyImpl.java       |   25 +-
 .../java/org/apache/phoenix/schema/PTable.java  |    7 +-
 .../org/apache/phoenix/schema/PTableImpl.java   |  118 +-
 .../apache/phoenix/schema/stat/PTableStats.java |   27 +-
 .../phoenix/schema/stat/PTableStatsImpl.java    |   37 +-
 .../schema/stat/StatisticsCollector.java        |  447 ++++++
 .../phoenix/schema/stat/StatisticsScanner.java  |  117 ++
 .../phoenix/schema/stat/StatisticsTable.java    |  168 +++
 .../phoenix/schema/stat/StatisticsTracker.java  |   62 +
 .../phoenix/schema/stat/StatisticsUtils.java    |   80 ++
 .../java/org/apache/phoenix/query/BaseTest.java |   17 +-
 .../phoenix/query/QueryServicesTestImpl.java    |    6 +-
 phoenix-protocol/src/main/MetaDataService.proto |   15 +-
 phoenix-protocol/src/main/PTable.proto          |    2 +-
 .../src/main/StatisticsCollect.proto            |   20 +
 pom.xml                                         |    2 +-
 59 files changed, 4829 insertions(+), 545 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayIT.java
index df0a76f..7863f5b 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayIT.java
@@ -143,10 +143,11 @@ public class ArrayIT extends BaseClientManagedTimeIT {
 		String query = "SELECT a_double_array, /* comment ok? */ b_string, a_float FROM table_with_array WHERE ?=organization_id and ?=a_float";
 		Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
 		props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB,
-				Long.toString(ts + 2)); // Execute at timestamp 2
+		        Long.toString(ts + 2)); // Execute at timestamp 2
 		Connection conn = DriverManager.getConnection(getUrl(), props);
+        analyzeTable(conn, TABLE_WITH_ARRAY);
 		try {
-			PreparedStatement statement = conn.prepareStatement(query);
+		    PreparedStatement statement = conn.prepareStatement(query);
 			statement.setString(1, tenantId);
 			statement.setFloat(2, 0.01f);
 			ResultSet rs = statement.executeQuery();
@@ -169,6 +170,12 @@ public class ArrayIT extends BaseClientManagedTimeIT {
 		}
 	}
 
+    private void analyzeTable(Connection conn, String tableWithArray) throws SQLException {
+        String analyse = "ANALYZE  "+tableWithArray;
+		PreparedStatement statement = conn.prepareStatement(analyse);
+        statement.execute();
+    }
+
 	@Test
 	public void testScanWithArrayInWhereClause() throws Exception {
 		long ts = nextTimestamp();
@@ -181,6 +188,7 @@ public class ArrayIT extends BaseClientManagedTimeIT {
 		props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB,
 				Long.toString(ts + 2)); // Execute at timestamp 2
 		Connection conn = DriverManager.getConnection(getUrl(), props);
+		analyzeTable(conn, TABLE_WITH_ARRAY);
 		try {
 			PreparedStatement statement = conn.prepareStatement(query);
 			statement.setString(1, tenantId);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseClientManagedTimeIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseClientManagedTimeIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseClientManagedTimeIT.java
index 79cb64b..8b47e11 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseClientManagedTimeIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseClientManagedTimeIT.java
@@ -52,7 +52,7 @@ import org.junit.experimental.categories.Category;
 public abstract class BaseClientManagedTimeIT extends BaseTest {
     private static String url;
     protected static PhoenixTestDriver driver;
-    private static final Configuration config = HBaseConfiguration.create(); 
+    protected static final Configuration config = HBaseConfiguration.create(); 
     private static boolean clusterInitialized = false;
     
     protected final static String getUrl() {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseParallelIteratorsRegionSplitterIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseParallelIteratorsRegionSplitterIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseParallelIteratorsRegionSplitterIT.java
index d2d3c7e..cfaa8ee 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseParallelIteratorsRegionSplitterIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseParallelIteratorsRegionSplitterIT.java
@@ -28,7 +28,6 @@ import java.util.Map;
 import java.util.Properties;
 
 import org.apache.phoenix.jdbc.PhoenixConnection;
-import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.TableRef;
 import org.apache.phoenix.util.PhoenixRuntime;
@@ -58,12 +57,7 @@ public class BaseParallelIteratorsRegionSplitterIT extends BaseClientManagedTime
     @BeforeClass
     @Shadower(classBeingShadowed = BaseClientManagedTimeIT.class)
     public static void doSetup() throws Exception {
-        int targetQueryConcurrency = 3;
-        int maxQueryConcurrency = 5;
         Map<String,String> props = Maps.newHashMapWithExpectedSize(3);
-        props.put(QueryServices.MAX_QUERY_CONCURRENCY_ATTRIB, Integer.toString(maxQueryConcurrency));
-        props.put(QueryServices.TARGET_QUERY_CONCURRENCY_ATTRIB, Integer.toString(targetQueryConcurrency));
-        props.put(QueryServices.MAX_INTRA_REGION_PARALLELIZATION_ATTRIB, Integer.toString(Integer.MAX_VALUE));
         // Must update config before starting server
         setUpTestDriver(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
     }
@@ -88,7 +82,8 @@ public class BaseParallelIteratorsRegionSplitterIT extends BaseClientManagedTime
 
     protected static TableRef getTableRef(Connection conn, long ts) throws SQLException {
         PhoenixConnection pconn = conn.unwrap(PhoenixConnection.class);
-        TableRef table = new TableRef(null,pconn.getMetaDataCache().getTable(new PTableKey(pconn.getTenantId(), STABLE_NAME)),ts, false);
+        TableRef table = new TableRef(null, pconn.getMetaDataCache().getTable(
+                new PTableKey(pconn.getTenantId(), STABLE_NAME)), ts, false);
         return table;
     }
     

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseQueryIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseQueryIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseQueryIT.java
index 4263dd2..9a66499 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseQueryIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseQueryIT.java
@@ -70,16 +70,12 @@ public abstract class BaseQueryIT extends BaseClientManagedTimeIT {
     @BeforeClass
     @Shadower(classBeingShadowed = BaseClientManagedTimeIT.class)
     public static void doSetup() throws Exception {
-        int targetQueryConcurrency = 2;
-        int maxQueryConcurrency = 3;
         Map<String,String> props = Maps.newHashMapWithExpectedSize(5);
-        props.put(QueryServices.QUEUE_SIZE_ATTRIB, Integer.toString(100));
-        props.put(QueryServices.MAX_QUERY_CONCURRENCY_ATTRIB, Integer.toString(maxQueryConcurrency));
-        props.put(QueryServices.TARGET_QUERY_CONCURRENCY_ATTRIB, Integer.toString(targetQueryConcurrency));
+        props.put(QueryServices.QUEUE_SIZE_ATTRIB, Integer.toString(5000));
         props.put(IndexWriterUtils.HTABLE_THREAD_KEY, Integer.toString(100));
         // Make a small batch size to test multiple calls to reserve sequences
         props.put(QueryServices.SEQUENCE_CACHE_SIZE_ATTRIB, Long.toString(BATCH_SIZE));
-        
+        props.put(QueryServices.HISTOGRAM_BYTE_DEPTH_CONF_KEY, Integer.toString(20));
         // Must update config before starting server
         setUpTestDriver(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewIT.java
index 5d8df0f..1cc9207 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewIT.java
@@ -27,7 +27,6 @@ import java.sql.DriverManager;
 import java.sql.ResultSet;
 import java.util.Map;
 
-import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.junit.BeforeClass;
@@ -43,7 +42,6 @@ public class BaseViewIT extends BaseHBaseManagedTimeIT {
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
         // Don't split intra region so we can more easily know that the n-way parallelization is for the explain plan
-        props.put(QueryServices.MAX_INTRA_REGION_PARALLELIZATION_ATTRIB, Integer.toString(1));
         // Must update config before starting server
         setUpTestDriver(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/it/java/org/apache/phoenix/end2end/DefaultParallelIteratorsRegionSplitterIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DefaultParallelIteratorsRegionSplitterIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DefaultParallelIteratorsRegionSplitterIT.java
index cd8f1fb..dd1dc8b 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DefaultParallelIteratorsRegionSplitterIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DefaultParallelIteratorsRegionSplitterIT.java
@@ -22,6 +22,7 @@ import static org.junit.Assert.assertEquals;
 
 import java.sql.Connection;
 import java.sql.DriverManager;
+import java.sql.PreparedStatement;
 import java.sql.SQLException;
 import java.util.Collections;
 import java.util.Comparator;
@@ -38,7 +39,6 @@ import org.apache.phoenix.iterate.DefaultParallelIteratorRegionSplitter;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.parse.HintNode;
-import org.apache.phoenix.query.ConnectionQueryServices;
 import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.schema.PDataType;
 import org.apache.phoenix.schema.TableRef;
@@ -85,64 +85,61 @@ public class DefaultParallelIteratorsRegionSplitterIT extends BaseParallelIterat
     public void testGetSplits() throws Exception {
         long ts = nextTimestamp();
         initTableValues(ts);
-        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + ts;
+        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + ts + 2;
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(url, props);
-
+        PreparedStatement stmt = conn.prepareStatement("ANALYZE STABLE");
+        stmt.execute();
         Scan scan = new Scan();
         
         // number of regions > target query concurrency
         scan.setStartRow(K1);
         scan.setStopRow(K12);
         List<KeyRange> keyRanges = getSplits(conn, ts, scan);
-        assertEquals("Unexpected number of splits: " + keyRanges, 5, keyRanges.size());
-        assertEquals(newKeyRange(KeyRange.UNBOUND, K3), keyRanges.get(0));
-        assertEquals(newKeyRange(K3, K4), keyRanges.get(1));
-        assertEquals(newKeyRange(K4, K9), keyRanges.get(2));
-        assertEquals(newKeyRange(K9, K11), keyRanges.get(3));
-        assertEquals(newKeyRange(K11, KeyRange.UNBOUND), keyRanges.get(4));
+        assertEquals("Unexpected number of splits: " + keyRanges, 7, keyRanges.size());
+        assertEquals(newKeyRange(KeyRange.UNBOUND, KMIN), keyRanges.get(0));
+        assertEquals(newKeyRange(KMIN, K3), keyRanges.get(1));
+        assertEquals(newKeyRange(K3, K4), keyRanges.get(2));
+        assertEquals(newKeyRange(K4, K9), keyRanges.get(3));
+        assertEquals(newKeyRange(K9, K11), keyRanges.get(4));
+        assertEquals(newKeyRange(K11, KMAX), keyRanges.get(5));
+        assertEquals(newKeyRange(KMAX,  KeyRange.UNBOUND), keyRanges.get(6));
         
-        // (number of regions / 2) > target query concurrency
         scan.setStartRow(K3);
         scan.setStopRow(K6);
         keyRanges = getSplits(conn, ts, scan);
-        assertEquals("Unexpected number of splits: " + keyRanges, 3, keyRanges.size());
+        assertEquals("Unexpected number of splits: " + keyRanges, 2, keyRanges.size());
         // note that we get a single split from R2 due to small key space
         assertEquals(newKeyRange(K3, K4), keyRanges.get(0));
-        assertEquals(newKeyRange(K4, K6), keyRanges.get(1));
-        assertEquals(newKeyRange(K6, K9), keyRanges.get(2));
+        assertEquals(newKeyRange(K4, K9), keyRanges.get(1));
         
-        // (number of regions / 2) <= target query concurrency
         scan.setStartRow(K5);
         scan.setStopRow(K6);
         keyRanges = getSplits(conn, ts, scan);
-        assertEquals("Unexpected number of splits: " + keyRanges, 3, keyRanges.size());
-        assertEquals(newKeyRange(K4, K5), keyRanges.get(0));
-        assertEquals(newKeyRange(K5, K6), keyRanges.get(1));
-        assertEquals(newKeyRange(K6, K9), keyRanges.get(2));
+        assertEquals("Unexpected number of splits: " + keyRanges, 1, keyRanges.size());
+        assertEquals(newKeyRange(K4, K9), keyRanges.get(0));
         conn.close();
     }
 
     @Test
-    public void testGetLowerUnboundSplits() throws Exception {
+    public void testGetLowerUnboundSplits() throws Throwable {
         long ts = nextTimestamp();
         initTableValues(ts);
-        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + ts;
+        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB;
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(url, props);
-
+        PreparedStatement stmt = conn.prepareStatement("ANALYZE STABLE");
+        stmt.execute();
+        // The query would use all the split points here
+        conn.createStatement().executeQuery("SELECT * FROM STABLE");
+        conn.close();
         Scan scan = new Scan();
-        
-        ConnectionQueryServices services = driver.getConnectionQueryServices(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES));
-        TableRef table = getTableRef(conn,ts);
-        services.getStatsManager().updateStats(table);
         scan.setStartRow(HConstants.EMPTY_START_ROW);
         scan.setStopRow(K1);
         List<KeyRange> keyRanges = getSplits(conn, ts, scan);
-        assertEquals("Unexpected number of splits: " + keyRanges, 3, keyRanges.size());
-        assertEquals(newKeyRange(KeyRange.UNBOUND, new byte[] {'7'}), keyRanges.get(0));
-        assertEquals(newKeyRange(new byte[] {'7'}, new byte[] {'M'}), keyRanges.get(1));
-        assertEquals(newKeyRange(new byte[] {'M'}, K3), keyRanges.get(2));
+        assertEquals("Unexpected number of splits: " + keyRanges, 2, keyRanges.size());
+        assertEquals(newKeyRange(KeyRange.UNBOUND, KMIN), keyRanges.get(0));
+        assertEquals(newKeyRange(KMIN, K3), keyRanges.get(1));
     }
 
     private static KeyRange newKeyRange(byte[] lowerRange, byte[] upperRange) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/it/java/org/apache/phoenix/end2end/GuidePostsLifeCycleIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/GuidePostsLifeCycleIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/GuidePostsLifeCycleIT.java
new file mode 100644
index 0000000..7645040
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/GuidePostsLifeCycleIT.java
@@ -0,0 +1,148 @@
+/*
+ * 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.STABLE_NAME;
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Properties;
+
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.compile.SequenceManager;
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.iterate.DefaultParallelIteratorRegionSplitter;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixStatement;
+import org.apache.phoenix.parse.HintNode;
+import org.apache.phoenix.query.KeyRange;
+import org.apache.phoenix.schema.PTableKey;
+import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(HBaseManagedTimeTest.class)
+public class GuidePostsLifeCycleIT extends BaseHBaseManagedTimeIT {
+
+    protected static final byte[] KMIN  = new byte[] {'!'};
+    protected static final byte[] KMIN2  = new byte[] {'.'};
+    protected static final byte[] K1  = new byte[] {'a'};
+    protected static final byte[] K3  = new byte[] {'c'};
+    protected static final byte[] K4  = new byte[] {'d'};
+    protected static final byte[] K5  = new byte[] {'e'};
+    protected static final byte[] K6  = new byte[] {'f'};
+    protected static final byte[] K9  = new byte[] {'i'};
+    protected static final byte[] K11 = new byte[] {'k'};
+    protected static final byte[] K12 = new byte[] {'l'};
+    protected static final byte[] KMAX  = new byte[] {'~'};
+    protected static final byte[] KMAX2  = new byte[] {'z'};
+    protected static final byte[] KR = new byte[] { 'r' };
+    protected static final byte[] KP = new byte[] { 'p' };
+
+    private static List<KeyRange> getSplits(Connection conn, final Scan scan) throws SQLException {
+        TableRef tableRef = getTableRef(conn);
+        PhoenixConnection pconn = conn.unwrap(PhoenixConnection.class);
+        final List<HRegionLocation> regions = pconn.getQueryServices().getAllTableRegions(
+                tableRef.getTable().getPhysicalName().getBytes());
+        PhoenixStatement statement = new PhoenixStatement(pconn);
+        StatementContext context = new StatementContext(statement, null, scan, new SequenceManager(statement));
+        DefaultParallelIteratorRegionSplitter splitter = new DefaultParallelIteratorRegionSplitter(context, tableRef,
+                HintNode.EMPTY_HINT_NODE) {
+            @Override
+            protected List<HRegionLocation> getAllRegions() throws SQLException {
+                return DefaultParallelIteratorRegionSplitter.filterRegions(regions, scan.getStartRow(),
+                        scan.getStopRow());
+            }
+        };
+        List<KeyRange> keyRanges = splitter.getSplits();
+        Collections.sort(keyRanges, new Comparator<KeyRange>() {
+            @Override
+            public int compare(KeyRange o1, KeyRange o2) {
+                return Bytes.compareTo(o1.getLowerRange(), o2.getLowerRange());
+            }
+        });
+        return keyRanges;
+    }
+
+    // This test ensures that as we keep adding new records the splits gets updated
+    @Test
+    public void testGuidePostsLifeCycle() throws Exception {
+        byte[][] splits = new byte[][] { K3, K9, KR };
+        ensureTableCreated(getUrl(), STABLE_NAME, splits);
+        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB;
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(url, props);
+        PreparedStatement stmt = conn.prepareStatement("ANALYZE STABLE");
+        stmt.execute();
+        Scan scan = new Scan();
+        List<KeyRange> keyRanges = getSplits(conn, scan);
+        assertEquals(4, keyRanges.size());
+        upsert(new byte[][] { KMIN, K4, K11 });
+        stmt = conn.prepareStatement("ANALYZE STABLE");
+        stmt.execute();
+        keyRanges = getSplits(conn, scan);
+        assertEquals(7, keyRanges.size());
+        upsert(new byte[][] { KMIN2, K5, K12 });
+        stmt = conn.prepareStatement("ANALYZE STABLE");
+        stmt.execute();
+        keyRanges = getSplits(conn, scan);
+        assertEquals(10, keyRanges.size());
+        upsert(new byte[][] { K1, K6, KP });
+        stmt = conn.prepareStatement("ANALYZE STABLE");
+        stmt.execute();
+        keyRanges = getSplits(conn, scan);
+        assertEquals(13, keyRanges.size());
+        conn.close();
+    }
+
+    protected void upsert( byte[][] val) throws Exception {
+        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB;
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(url, props);
+        PreparedStatement stmt = conn.prepareStatement("upsert into " + STABLE_NAME + " VALUES (?, ?)");
+        stmt.setString(1, new String(val[0]));
+        stmt.setInt(2, 1);
+        stmt.execute();
+        stmt.setString(1, new String(val[1]));
+        stmt.setInt(2, 2);
+        stmt.execute();
+        stmt.setString(1, new String(val[2]));
+        stmt.setInt(2, 3);
+        stmt.execute();
+        conn.commit();
+        conn.close();
+    }
+    
+    protected static TableRef getTableRef(Connection conn) throws SQLException {
+        PhoenixConnection pconn = conn.unwrap(PhoenixConnection.class);
+        TableRef table = new TableRef(null, pconn.getMetaDataCache().getTable(
+                new PTableKey(pconn.getTenantId(), STABLE_NAME)), System.currentTimeMillis(), false);
+        return table;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinIT.java
index 05f2837..1d2ca4c 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinIT.java
@@ -82,8 +82,6 @@ public class HashJoinIT extends BaseHBaseManagedTimeIT {
     @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(3);
-        // Don't split intra region so we can more easily know that the n-way parallelization is for the explain plan
-        props.put(QueryServices.MAX_INTRA_REGION_PARALLELIZATION_ATTRIB, Integer.toString(1));
         // Forces server cache to be used
         props.put(QueryServices.INDEX_MUTATE_BATCH_SIZE_THRESHOLD_ATTRIB, Integer.toString(2));
         // Must update config before starting server

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/it/java/org/apache/phoenix/end2end/KeyOnlyIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/KeyOnlyIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/KeyOnlyIT.java
index 813f4b8..4b0d07f 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/KeyOnlyIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/KeyOnlyIT.java
@@ -23,12 +23,29 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
+import java.io.IOException;
 import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
 import java.util.Properties;
 
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.compile.SequenceManager;
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.iterate.DefaultParallelIteratorRegionSplitter;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixStatement;
+import org.apache.phoenix.parse.HintNode;
+import org.apache.phoenix.query.KeyRange;
+import org.apache.phoenix.schema.PTableKey;
+import org.apache.phoenix.schema.TableRef;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.junit.Test;
@@ -45,6 +62,7 @@ public class KeyOnlyIT extends BaseClientManagedTimeIT {
         
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts+5));
         Connection conn5 = DriverManager.getConnection(getUrl(), props);
+        analyzeTable(conn5, KEYONLY_NAME);
         String query = "SELECT i1, i2 FROM KEYONLY";
         PreparedStatement statement = conn5.prepareStatement(query);
         ResultSet rs = statement.executeQuery();
@@ -55,6 +73,9 @@ public class KeyOnlyIT extends BaseClientManagedTimeIT {
         assertEquals(3, rs.getInt(1));
         assertEquals(4, rs.getInt(2));
         assertFalse(rs.next());
+        Scan scan = new Scan();
+        List<KeyRange> splits = getSplits(conn5, ts, scan);
+        assertEquals(3, splits.size());
         conn5.close();
         
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts+6));
@@ -76,6 +97,7 @@ public class KeyOnlyIT extends BaseClientManagedTimeIT {
         
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts+8));
         Connection conn8 = DriverManager.getConnection(getUrl(), props);
+        analyzeTable(conn8, KEYONLY_NAME);
         query = "SELECT i1 FROM KEYONLY";
         statement = conn8.prepareStatement(query);
         rs = statement.executeQuery();
@@ -113,6 +135,7 @@ public class KeyOnlyIT extends BaseClientManagedTimeIT {
         
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts+5));
         Connection conn5 = DriverManager.getConnection(getUrl(), props);
+        analyzeTable(conn5, KEYONLY_NAME);
         String query = "SELECT i1 FROM KEYONLY WHERE i1 < 2 or i1 = 3";
         PreparedStatement statement = conn5.prepareStatement(query);
         ResultSet rs = statement.executeQuery();
@@ -142,5 +165,41 @@ public class KeyOnlyIT extends BaseClientManagedTimeIT {
         conn.commit();
         conn.close();
     }
-        
+
+    private void analyzeTable(Connection conn, String tableName) throws IOException, SQLException {
+        String query = "ANALYZE " + tableName;
+        conn.createStatement().execute(query);
+    }
+    
+    private static TableRef getTableRef(Connection conn, long ts) throws SQLException {
+        PhoenixConnection pconn = conn.unwrap(PhoenixConnection.class);
+        TableRef table = new TableRef(null, pconn.getMetaDataCache().getTable(
+                new PTableKey(pconn.getTenantId(), KEYONLY_NAME)), ts, false);
+        return table;
+    }
+
+    private static List<KeyRange> getSplits(Connection conn, long ts, final Scan scan) throws SQLException {
+        TableRef tableRef = getTableRef(conn, ts);
+        PhoenixConnection pconn = conn.unwrap(PhoenixConnection.class);
+        final List<HRegionLocation> regions = pconn.getQueryServices().getAllTableRegions(
+                tableRef.getTable().getPhysicalName().getBytes());
+        PhoenixStatement statement = new PhoenixStatement(pconn);
+        StatementContext context = new StatementContext(statement, null, scan, new SequenceManager(statement));
+        DefaultParallelIteratorRegionSplitter splitter = new DefaultParallelIteratorRegionSplitter(context, tableRef,
+                HintNode.EMPTY_HINT_NODE) {
+            @Override
+            protected List<HRegionLocation> getAllRegions() throws SQLException {
+                return DefaultParallelIteratorRegionSplitter.filterRegions(regions, scan.getStartRow(),
+                        scan.getStopRow());
+            }
+        };
+        List<KeyRange> keyRanges = splitter.getSplits();
+        Collections.sort(keyRanges, new Comparator<KeyRange>() {
+            @Override
+            public int compare(KeyRange o1, KeyRange o2) {
+                return Bytes.compareTo(o1.getLowerRange(), o2.getLowerRange());
+            }
+        });
+        return keyRanges;
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/it/java/org/apache/phoenix/end2end/MultiCfQueryExecIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MultiCfQueryExecIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MultiCfQueryExecIT.java
index b5e0ef4..ebf03d0 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MultiCfQueryExecIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MultiCfQueryExecIT.java
@@ -22,13 +22,30 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
+import java.io.IOException;
 import java.math.BigDecimal;
 import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
 import java.util.Properties;
 
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.compile.SequenceManager;
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.iterate.DefaultParallelIteratorRegionSplitter;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixStatement;
+import org.apache.phoenix.parse.HintNode;
+import org.apache.phoenix.query.KeyRange;
+import org.apache.phoenix.schema.PTableKey;
+import org.apache.phoenix.schema.TableRef;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.junit.Test;
@@ -74,7 +91,12 @@ public class MultiCfQueryExecIT extends BaseClientManagedTimeIT {
         stmt.setLong(7, 22222);
         stmt.execute();
     }
-    
+
+    private void analyzeTable(Connection conn, String tableName) throws IOException, SQLException {
+        String query = "ANALYZE " + tableName;
+        conn.createStatement().execute(query);
+    }
+
     @Test
     public void testConstantCount() throws Exception {
         long ts = nextTimestamp();
@@ -84,6 +106,7 @@ public class MultiCfQueryExecIT extends BaseClientManagedTimeIT {
         Connection conn = DriverManager.getConnection(url, props);
         try {
             initTableValues(ts);
+            analyzeTable(conn, "MULTI_CF");
             PreparedStatement statement = conn.prepareStatement(query);
             ResultSet rs = statement.executeQuery();
             assertTrue(rs.next());
@@ -103,6 +126,7 @@ public class MultiCfQueryExecIT extends BaseClientManagedTimeIT {
         Connection conn = DriverManager.getConnection(url, props);
         try {
             initTableValues(ts);
+            analyzeTable(conn, "MULTI_CF");
             PreparedStatement statement = conn.prepareStatement(query);
             ResultSet rs = statement.executeQuery();
             assertTrue(rs.next());
@@ -123,6 +147,7 @@ public class MultiCfQueryExecIT extends BaseClientManagedTimeIT {
         Connection conn = DriverManager.getConnection(url, props);
         try {
             initTableValues(ts);
+            analyzeTable(conn, "MULTI_CF");
             PreparedStatement statement = conn.prepareStatement(query);
             ResultSet rs = statement.executeQuery();
             assertTrue(rs.next());
@@ -143,12 +168,24 @@ public class MultiCfQueryExecIT extends BaseClientManagedTimeIT {
         Connection conn = DriverManager.getConnection(url, props);
         try {
             initTableValues(ts);
+            analyzeTable(conn, "MULTI_CF");
             PreparedStatement statement = conn.prepareStatement(query);
             ResultSet rs = statement.executeQuery();
             assertTrue(rs.next());
             assertEquals(2222, rs.getLong(1));
             assertEquals(22222, rs.getLong(2));
             assertFalse(rs.next());
+            Scan scan = new Scan();
+            // See if F has splits in it
+            scan.addFamily(Bytes.toBytes("E"));
+            List<KeyRange> splits = getSplits(conn, ts, scan);
+            assertEquals(3, splits.size());
+            scan = new Scan();
+            // See if G has splits in it
+            scan.addFamily(Bytes.toBytes("G"));
+            splits = getSplits(conn, ts, scan);
+            // We get splits from different CF
+            assertEquals(3, splits.size());
         } finally {
             conn.close();
         }
@@ -163,6 +200,7 @@ public class MultiCfQueryExecIT extends BaseClientManagedTimeIT {
         Connection conn = DriverManager.getConnection(url, props);
         try {
             initTableValues(ts);
+            analyzeTable(conn, "MULTI_CF");
             PreparedStatement statement = conn.prepareStatement(query);
             ResultSet rs = statement.executeQuery();
             assertTrue(rs.next());
@@ -178,11 +216,11 @@ public class MultiCfQueryExecIT extends BaseClientManagedTimeIT {
     public void testDefaultCFToDisambiguate() throws Exception {
         long ts = nextTimestamp();
         initTableValues(ts);
-        
         String ddl = "ALTER TABLE multi_cf ADD response_time BIGINT";
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 3);
         Connection conn = DriverManager.getConnection(url);
         conn.createStatement().execute(ddl);
+        analyzeTable(conn, "MULTI_CF");
         conn.close();
        
         String dml = "upsert into " +
@@ -195,7 +233,7 @@ public class MultiCfQueryExecIT extends BaseClientManagedTimeIT {
         conn.createStatement().execute(dml);
         conn.commit();
         conn.close();
-
+        analyzeTable(conn, "MULTI_CF");
         String query = "SELECT ID,RESPONSE_TIME from multi_cf where RESPONSE_TIME = 333";
         url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
         conn = DriverManager.getConnection(url);
@@ -220,6 +258,7 @@ public class MultiCfQueryExecIT extends BaseClientManagedTimeIT {
         Connection conn = DriverManager.getConnection(url, props);
         try {
             initTableValues(ts);
+            analyzeTable(conn, "MULTI_CF");
             PreparedStatement statement = conn.prepareStatement(query);
             ResultSet rs = statement.executeQuery();
             assertTrue(rs.next());
@@ -230,5 +269,36 @@ public class MultiCfQueryExecIT extends BaseClientManagedTimeIT {
             conn.close();
         }
     }
-    
+
+    private static TableRef getTableRef(Connection conn, long ts) throws SQLException {
+        PhoenixConnection pconn = conn.unwrap(PhoenixConnection.class);
+        TableRef table = new TableRef(null, pconn.getMetaDataCache().getTable(
+                new PTableKey(pconn.getTenantId(), "MULTI_CF")), ts, false);
+        return table;
+    }
+
+    private static List<KeyRange> getSplits(Connection conn, long ts, final Scan scan) throws SQLException {
+        TableRef tableRef = getTableRef(conn, ts);
+        PhoenixConnection pconn = conn.unwrap(PhoenixConnection.class);
+        final List<HRegionLocation> regions = pconn.getQueryServices().getAllTableRegions(
+                tableRef.getTable().getPhysicalName().getBytes());
+        PhoenixStatement statement = new PhoenixStatement(pconn);
+        StatementContext context = new StatementContext(statement, null, scan, new SequenceManager(statement));
+        DefaultParallelIteratorRegionSplitter splitter = new DefaultParallelIteratorRegionSplitter(context, tableRef,
+                HintNode.EMPTY_HINT_NODE) {
+            @Override
+            protected List<HRegionLocation> getAllRegions() throws SQLException {
+                return DefaultParallelIteratorRegionSplitter.filterRegions(regions, scan.getStartRow(),
+                        scan.getStopRow());
+            }
+        };
+        List<KeyRange> keyRanges = splitter.getSplits();
+        Collections.sort(keyRanges, new Comparator<KeyRange>() {
+            @Override
+            public int compare(KeyRange o1, KeyRange o2) {
+                return Bytes.compareTo(o1.getLowerRange(), o2.getLowerRange());
+            }
+        });
+        return keyRanges;
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java
index b9cd477..f243562 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java
@@ -113,6 +113,10 @@ public class QueryDatabaseMetaDataIT extends BaseClientManagedTimeIT {
         assertEquals(rs.getString("TABLE_NAME"),TYPE_SEQUENCE);
         assertEquals(PTableType.SYSTEM.toString(), rs.getString("TABLE_TYPE"));
         assertTrue(rs.next());
+        assertEquals(rs.getString("TABLE_SCHEM"),SYSTEM_CATALOG_SCHEMA);
+        assertEquals(rs.getString("TABLE_NAME"),PhoenixDatabaseMetaData.SYSTEM_STATS_TABLE);
+        assertEquals(PTableType.SYSTEM.toString(), rs.getString("TABLE_TYPE"));
+        assertTrue(rs.next());
         assertEquals(rs.getString("TABLE_SCHEM"),null);
         assertEquals(rs.getString("TABLE_NAME"),ATABLE_NAME);
         assertEquals(PTableType.TABLE.toString(), rs.getString("TABLE_TYPE"));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryIT.java
index c9ec25e..674009b 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryIT.java
@@ -38,17 +38,18 @@ import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+import java.io.IOException;
 import java.sql.Connection;
 import java.sql.Date;
 import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
+import java.sql.SQLException;
 import java.sql.Time;
 import java.sql.Timestamp;
 import java.util.Arrays;
 import java.util.Properties;
 
-import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
@@ -59,7 +60,6 @@ import org.apache.phoenix.schema.ConstraintViolationException;
 import org.apache.phoenix.schema.PDataType;
 import org.apache.phoenix.schema.SequenceNotFoundException;
 import org.apache.phoenix.util.ByteUtil;
-import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.junit.Test;
@@ -102,6 +102,11 @@ public class QueryIT extends BaseQueryIT {
         stmt.setInt(3, -10);
         stmt.execute();
         upsertConn.close();
+        url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 6);
+        props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        upsertConn = DriverManager.getConnection(url, props);
+        analyzeTable(upsertConn, "ATABLE");
+        upsertConn.close();
 
         String query = "SELECT entity_id FROM aTable WHERE organization_id=? and a_integer >= ?";
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
@@ -210,6 +215,9 @@ public class QueryIT extends BaseQueryIT {
         stmt.setString(2, ROW5);
         stmt.setString(3, value);
         stmt.execute(); // should commit too
+        Connection conn1 = DriverManager.getConnection(getUrl(), props);
+        analyzeTable(conn1, "ATABLE");
+        conn1.close();
         upsertConn.close();
         
         String query = "SELECT a_string, b_string FROM aTable WHERE organization_id=? and a_integer = 5";
@@ -254,6 +262,9 @@ public class QueryIT extends BaseQueryIT {
         stmt.setString(2, ROW4);
         stmt.setInt(3, 5);
         stmt.execute(); // should commit too
+        Connection conn1 = DriverManager.getConnection(getUrl(), props);
+        analyzeTable(conn1, "ATABLE");
+        conn1.close();
         upsertConn.close();
 
         // Override value again, but should be ignored since it's past the SCN
@@ -389,7 +400,9 @@ public class QueryIT extends BaseQueryIT {
         byte[] ts1 = PDataType.TIMESTAMP.toBytes(tsValue1);
         stmt.setTimestamp(3, tsValue1);
         stmt.execute();
-
+        Connection conn1 = DriverManager.getConnection(getUrl(), props);
+        analyzeTable(conn1, "ATABLE");
+        conn1.close();
         updateStmt = 
             "upsert into " +
             "ATABLE(" +
@@ -408,7 +421,10 @@ public class QueryIT extends BaseQueryIT {
         stmt.setTime(4, new Time(tsValue2.getTime()));
         stmt.execute();
         upsertConn.close();
-        
+        conn1 = DriverManager.getConnection(getUrl(), props);
+        analyzeTable(conn1, "ATABLE");
+        conn1.close();
+        analyzeTable(upsertConn, "ATABLE");
         assertTrue(compare(CompareOp.GREATER, new ImmutableBytesWritable(ts2), new ImmutableBytesWritable(ts1)));
         assertFalse(compare(CompareOp.GREATER, new ImmutableBytesWritable(ts1), new ImmutableBytesWritable(ts1)));
 
@@ -734,7 +750,7 @@ public class QueryIT extends BaseQueryIT {
             assertTrue(rs.next());
             assertEquals(A_VALUE, rs.getString(1));
             assertEquals(E_VALUE, rs.getString(2));
-           assertEquals(1, rs.getLong(3));
+            assertEquals(1, rs.getLong(3));
             assertFalse(rs.next());
             
             byte[] tableName = Bytes.toBytes(ATABLE_NAME);
@@ -801,6 +817,10 @@ public class QueryIT extends BaseQueryIT {
         Connection conn = DriverManager.getConnection(getUrl(), props);
         conn.setAutoCommit(true);
         conn.createStatement().execute("UPSERT INTO atable(organization_id,entity_id,a_integer) VALUES('" + getOrganizationId() + "','" + ROW3 + "',NULL)");
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 6));
+        Connection conn1 = DriverManager.getConnection(getUrl(), props);
+        analyzeTable(conn1, "ATABLE");
+        conn1.close();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 5));
         conn = DriverManager.getConnection(getUrl(), props);
         try {
@@ -816,6 +836,10 @@ public class QueryIT extends BaseQueryIT {
         conn = DriverManager.getConnection(getUrl(), props);
         conn.setAutoCommit(true);
         conn.createStatement().execute("UPSERT INTO atable(organization_id,entity_id,a_integer) SELECT organization_id, entity_id, null FROM atable");
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 6));
+        conn1 = DriverManager.getConnection(getUrl(), props);
+        analyzeTable(conn1, "ATABLE");
+        conn1.close();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 9));
         conn = DriverManager.getConnection(getUrl(), props);
         try {
@@ -829,4 +853,9 @@ public class QueryIT extends BaseQueryIT {
             conn.close();
         }
     }
+
+    private void analyzeTable(Connection conn, String tableName) throws IOException, SQLException {
+        String query = "ANALYZE " + tableName;
+        conn.createStatement().execute(query);
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/it/java/org/apache/phoenix/end2end/ReverseScanIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ReverseScanIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ReverseScanIT.java
index e87d290..805ec3b 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ReverseScanIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ReverseScanIT.java
@@ -38,7 +38,6 @@ import org.apache.hadoop.hbase.TableNotFoundException;
 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.QueryServices;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.TestUtil;
@@ -54,7 +53,6 @@ public class ReverseScanIT extends BaseClientManagedTimeIT {
     @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
-        props.put(QueryServices.MAX_INTRA_REGION_PARALLELIZATION_ATTRIB, Integer.toString(1));
         setUpTestDriver(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
         // Ensures our split points will be used
         // TODO: do deletePriorTables before test?

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipRangeParallelIteratorRegionSplitterIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipRangeParallelIteratorRegionSplitterIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipRangeParallelIteratorRegionSplitterIT.java
index 93abfb9..d32441b 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipRangeParallelIteratorRegionSplitterIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipRangeParallelIteratorRegionSplitterIT.java
@@ -45,7 +45,6 @@ import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.parse.HintNode;
 import org.apache.phoenix.query.KeyRange;
-import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.ColumnRef;
 import org.apache.phoenix.schema.PDataType;
 import org.apache.phoenix.schema.PDatum;
@@ -81,12 +80,8 @@ public class SkipRangeParallelIteratorRegionSplitterIT extends BaseClientManaged
     private static final String DDL = "CREATE TABLE " + TABLE_NAME + " (id char(3) NOT NULL PRIMARY KEY, \"value\" integer)";
     private static final byte[] Ka1A = Bytes.toBytes("a1A");
     private static final byte[] Ka1B = Bytes.toBytes("a1B");
-    private static final byte[] Ka1C = Bytes.toBytes("a1C");
-    private static final byte[] Ka1D = Bytes.toBytes("a1D");
     private static final byte[] Ka1E = Bytes.toBytes("a1E");
-    private static final byte[] Ka1F = Bytes.toBytes("a1F");
     private static final byte[] Ka1G = Bytes.toBytes("a1G");
-    private static final byte[] Ka1H = Bytes.toBytes("a1H");
     private static final byte[] Ka1I = Bytes.toBytes("a1I");
     private static final byte[] Ka2A = Bytes.toBytes("a2A");
 
@@ -109,11 +104,13 @@ public class SkipRangeParallelIteratorRegionSplitterIT extends BaseClientManaged
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(url, props);
         PhoenixConnection pconn = conn.unwrap(PhoenixConnection.class);
-        TableRef tableRef = new TableRef(null,pconn.getMetaDataCache().getTable(new PTableKey(pconn.getTenantId(), TABLE_NAME)),ts, false);
-        List<HRegionLocation> regions = pconn.getQueryServices().getAllTableRegions(tableRef.getTable().getPhysicalName().getBytes());
         
-        conn.close();
         initTableValues();
+        PreparedStatement stmt = conn.prepareStatement("ANALYZE "+TABLE_NAME);
+        stmt.execute();
+        conn.close();
+        TableRef tableRef = new TableRef(null,pconn.getMetaDataCache().getTable(new PTableKey(pconn.getTenantId(), TABLE_NAME)),ts, false);
+        List<HRegionLocation> regions = pconn.getQueryServices().getAllTableRegions(tableRef.getTable().getPhysicalName().getBytes());
         List<KeyRange> ranges = getSplits(tableRef, scan, regions, scanRanges);
         assertEquals("Unexpected number of splits: " + ranges.size(), expectedSplits.size(), ranges.size());
         for (int i=0; i<expectedSplits.size(); i++) {
@@ -188,9 +185,7 @@ public class SkipRangeParallelIteratorRegionSplitterIT extends BaseClientManaged
                         }},
                     new int[] {1,1,1},
                     new KeyRange[] {
-                        getKeyRange(Ka1B, true, Ka1C, false),
-                        getKeyRange(Ka1C, true, Ka1D, false),
-                        getKeyRange(Ka1D, true, Ka1E, false),
+                        getKeyRange(Ka1B, true, Ka1E, false)
                 }));
         // Scan range spans third, split into 3 due to concurrency config.
         testCases.addAll(
@@ -204,9 +199,7 @@ public class SkipRangeParallelIteratorRegionSplitterIT extends BaseClientManaged
                         }},
                     new int[] {1,1,1},
                     new KeyRange[] {
-                        getKeyRange(Ka1B, true, Ka1C, false),
-                        getKeyRange(Ka1C, true, Ka1D, false),
-                        getKeyRange(Ka1D, true, Ka1E, false),
+                        getKeyRange(Ka1B, true, Ka1E, false),
                 }));
         // Scan range spans 2 ranges, split into 4 due to concurrency config.
         testCases.addAll(
@@ -220,10 +213,8 @@ public class SkipRangeParallelIteratorRegionSplitterIT extends BaseClientManaged
                         }},
                     new int[] {1,1,1},
                     new KeyRange[] {
-                        getKeyRange(Ka1E, true, Ka1F, false),
-                        getKeyRange(Ka1F, true, Ka1G, false),
-                        getKeyRange(Ka1G, true, Ka1H, false),
-                        getKeyRange(Ka1H, true, Ka1I, false),
+                        getKeyRange(Ka1E, true, Ka1G, false),
+                        getKeyRange(Ka1G, true, Ka1I, false),
                 }));
         // Scan range spans more than 3 range, no split.
         testCases.addAll(
@@ -326,12 +317,7 @@ public class SkipRangeParallelIteratorRegionSplitterIT extends BaseClientManaged
     @BeforeClass
     @Shadower(classBeingShadowed = BaseClientManagedTimeIT.class)
     public static void doSetup() throws Exception {
-        int targetQueryConcurrency = 3;
-        int maxQueryConcurrency = 5;
         Map<String,String> props = Maps.newHashMapWithExpectedSize(3);
-        props.put(QueryServices.MAX_QUERY_CONCURRENCY_ATTRIB, Integer.toString(maxQueryConcurrency));
-        props.put(QueryServices.TARGET_QUERY_CONCURRENCY_ATTRIB, Integer.toString(targetQueryConcurrency));
-        props.put(QueryServices.MAX_INTRA_REGION_PARALLELIZATION_ATTRIB, Integer.toString(Integer.MAX_VALUE));
         // Must update config before starting server
         setUpTestDriver(getUrl(), new ReadOnlyProps(props.entrySet().iterator()));
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/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
new file mode 100644
index 0000000..407c128
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java
@@ -0,0 +1,231 @@
+package org.apache.phoenix.end2end;
+
+import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL;
+import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR;
+import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL_TERMINATOR;
+import static org.apache.phoenix.util.PhoenixRuntime.PHOENIX_TEST_DRIVER_URL_PARAM;
+import static org.apache.phoenix.util.TestUtil.LOCALHOST;
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+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.Map;
+import java.util.Properties;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.google.common.collect.Maps;
+
+@Category(HBaseManagedTimeTest.class)
+public class StatsCollectorIT extends BaseHBaseManagedTimeIT {
+    private static String url;
+    private static HBaseTestingUtility util;
+    private static int frequency = 4000;
+
+    @BeforeClass
+    public static void doSetup() throws Exception {
+        Configuration conf = HBaseConfiguration.create();
+        setUpConfigForMiniCluster(conf);
+        conf.setInt("hbase.client.retries.number", 2);
+        conf.setInt("hbase.client.pause", 5000);
+        conf.setLong(QueryServices.INDEX_FAILURE_HANDLING_REBUILD_OVERLAP_TIME_ATTRIB, 0);
+        util = new HBaseTestingUtility(conf);
+        util.startMiniCluster();
+        String clientPort = util.getConfiguration().get(QueryServices.ZOOKEEPER_PORT_ATTRIB);
+        url = JDBC_PROTOCOL + JDBC_PROTOCOL_SEPARATOR + LOCALHOST + JDBC_PROTOCOL_SEPARATOR + clientPort
+                + JDBC_PROTOCOL_TERMINATOR + PHOENIX_TEST_DRIVER_URL_PARAM;
+        int histogramDepth = 60;
+        Map<String, String> props = Maps.newHashMapWithExpectedSize(3);
+        props.put(QueryServices.HISTOGRAM_BYTE_DEPTH_CONF_KEY, Integer.toString(histogramDepth));
+        props.put(QueryServices.STATS_UPDATE_FREQ_MS_ATTRIB, Integer.toString(frequency));
+        driver = initAndRegisterDriver(url, new ReadOnlyProps(props.entrySet().iterator()));
+    }
+
+    @Test
+    public void testUpdateStatsForTheTable() throws Throwable {
+        Connection conn;
+        PreparedStatement stmt;
+        ResultSet rs;
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        // props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 10));
+        conn = DriverManager.getConnection(url, props);
+        conn.createStatement().execute(
+                "CREATE TABLE t ( 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)) \n");
+        String[] s;
+        Array array;
+        conn = upsertValues(props, "t");
+        // CAll the update statistics query here. If already major compaction has run this will not get executed.
+        stmt = conn.prepareStatement("ANALYZE T");
+        stmt.execute();
+        stmt = upsertStmt(conn, "t");
+        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 = DriverManager.getConnection(getUrl(), props);
+        // This analyze would not work
+        stmt = conn.prepareStatement("ANALYZE T");
+        stmt.execute();
+        rs = conn.createStatement().executeQuery("SELECT k FROM T");
+        assertTrue(rs.next());
+        conn.close();
+    }
+
+    @Test
+    public void testUpdateStatsWithMultipleTables() throws Throwable {
+        Connection conn;
+        PreparedStatement stmt;
+        ResultSet rs;
+        long ts = nextTimestamp();
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        // props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 10));
+        conn = DriverManager.getConnection(url, props);
+        conn.createStatement().execute(
+                "CREATE TABLE x ( 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)) \n");
+        conn.createStatement().execute(
+                "CREATE TABLE z ( 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)) \n");
+        String[] s;
+        Array array;
+        conn = upsertValues(props, "x");
+        conn = upsertValues(props, "z");
+        // CAll the update statistics query here
+        stmt = conn.prepareStatement("ANALYZE X");
+        stmt.execute();
+        stmt = conn.prepareStatement("ANALYZE Z");
+        stmt.execute();
+        stmt = upsertStmt(conn, "x");
+        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, "z");
+        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 = DriverManager.getConnection(getUrl(), props);
+        // This analyze would not work
+        stmt = conn.prepareStatement("ANALYZE Z");
+        stmt.execute();
+        rs = conn.createStatement().executeQuery("SELECT k FROM Z");
+        assertTrue(rs.next());
+        conn.close();
+    }
+
+    private Connection upsertValues(Properties props, String tableName) throws SQLException, IOException,
+            InterruptedException {
+        Connection conn;
+        PreparedStatement stmt;
+        // props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 30));
+        conn = DriverManager.getConnection(url, props);
+        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();
+        flush(tableName);
+        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();
+        flush(tableName);
+        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();
+        flush(tableName);
+        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();
+        flush(tableName);
+        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();
+        flush(tableName);
+        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();
+        flush(tableName);
+        return conn;
+    }
+
+    private void flush(String tableName) throws IOException, InterruptedException {
+        util.getHBaseAdmin().flush(tableName.toUpperCase());
+    }
+
+    private PreparedStatement upsertStmt(Connection conn, String tableName) throws SQLException {
+        PreparedStatement stmt;
+        stmt = conn.prepareStatement("UPSERT INTO " + tableName + " VALUES(?,?,?)");
+        return stmt;
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsManagerIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsManagerIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsManagerIT.java
index f19f776..b13379b 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsManagerIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsManagerIT.java
@@ -121,7 +121,7 @@ public class StatsManagerIT extends BaseParallelIteratorsRegionSplitterIT {
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + ts;
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(url, props);
-        TableRef table = getTableRef(conn,ts);
+        TableRef table = getTableRef(conn, ts);
 
         int updateFreq = 5;
         int maxAge = 10;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java
index 591efe1..655fe44 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java
@@ -474,6 +474,8 @@ public class TenantSpecificTablesDDLIT extends BaseTenantSpecificTablesIT {
             assertTrue(rs.next());
             assertTableMetaData(rs, SYSTEM_CATALOG_SCHEMA, TYPE_SEQUENCE, SYSTEM);
             assertTrue(rs.next());
+            assertTableMetaData(rs, SYSTEM_CATALOG_SCHEMA, PhoenixDatabaseMetaData.SYSTEM_STATS_TABLE, SYSTEM);
+            assertTrue(rs.next());
             assertTableMetaData(rs, null, PARENT_TABLE_NAME, TABLE);
             assertTrue(rs.next());
             assertTableMetaData(rs, null, PARENT_TABLE_NAME_NO_TENANT_TYPE_ID, TABLE);
@@ -538,6 +540,8 @@ public class TenantSpecificTablesDDLIT extends BaseTenantSpecificTablesIT {
             assertTrue(rs.next());
             assertTableMetaData(rs, PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA, PhoenixDatabaseMetaData.TYPE_SEQUENCE, PTableType.SYSTEM);
             assertTrue(rs.next());
+            assertTableMetaData(rs, SYSTEM_CATALOG_SCHEMA, PhoenixDatabaseMetaData.SYSTEM_STATS_TABLE, PTableType.SYSTEM);
+            assertTrue(rs.next());
             assertTableMetaData(rs, null, PARENT_TABLE_NAME, PTableType.TABLE);
             assertTrue(rs.next());
             assertTableMetaData(rs, null, PARENT_TABLE_NAME_NO_TENANT_TYPE_ID, PTableType.TABLE);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDMLIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDMLIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDMLIT.java
index 107ca34..dba4264 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDMLIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDMLIT.java
@@ -23,6 +23,7 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+import java.io.IOException;
 import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.ResultSet;
@@ -47,6 +48,7 @@ public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
             conn.createStatement().executeUpdate("upsert into " + TENANT_TABLE_NAME + " (id, tenant_col) values (2, 'Viva Las Vegas')");
             conn.commit();
             conn.close();
+            analyzeTable(conn, TENANT_TABLE_NAME);
             
             conn = nextConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL);
             ResultSet rs = conn.createStatement().executeQuery("select tenant_col from " + TENANT_TABLE_NAME + " where id = 1");
@@ -70,11 +72,11 @@ public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
             conn1.createStatement().executeUpdate("upsert into " + TENANT_TABLE_NAME + " values ('you','" + TENANT_TYPE_ID +"',2,'Viva Las Vegas')");
             conn1.commit();
             
-            
+            analyzeTable(conn1, TENANT_TABLE_NAME);
             conn2.setAutoCommit(true);
             conn2.createStatement().executeUpdate("upsert into " + TENANT_TABLE_NAME + " values ('them','" + TENANT_TYPE_ID + "',1,'Long Hair')");
             conn2.createStatement().executeUpdate("upsert into " + TENANT_TABLE_NAME + " values ('us','" + TENANT_TYPE_ID + "',2,'Black Hat')");
-
+            analyzeTable(conn2, TENANT_TABLE_NAME);
             conn2.close();            
             conn1.close();
             
@@ -96,6 +98,7 @@ public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
             conn2 = nextConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL2);
             conn2.createStatement().executeUpdate("upsert into " + TENANT_TABLE_NAME + " select * from " + TENANT_TABLE_NAME );
             conn2.commit();
+            analyzeTable(conn2, TENANT_TABLE_NAME);
             conn2.close();
             
             conn2 = nextConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL2);
@@ -112,6 +115,7 @@ public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
             conn2 = nextConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL2);
             conn2.setAutoCommit(true);;
             conn2.createStatement().executeUpdate("upsert into " + TENANT_TABLE_NAME + " select 'all', tenant_type_id, id, 'Big ' || tenant_col from " + TENANT_TABLE_NAME );
+            analyzeTable(conn2, TENANT_TABLE_NAME);
             conn2.close();
 
             conn2 = nextConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL2);
@@ -159,6 +163,7 @@ public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
             conn.createStatement().executeUpdate("upsert into " + TENANT_TABLE_NAME + " (id, tenant_col) values (1, 'Cheap Sunglasses')");
             conn.createStatement().executeUpdate("upsert into " + TENANT_TABLE_NAME + " (id, tenant_col) values (2, 'Viva Las Vegas')");
             conn.commit();
+            analyzeTable(conn, TENANT_TABLE_NAME);
             conn.close();
             
             conn = nextConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL);
@@ -185,6 +190,7 @@ public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
             conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('AC/DC', 'abc', 1, 'Bon Scott')");
             conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('" + TENANT_ID + "', '" + TENANT_TYPE_ID + "', 1, 'Billy Gibbons')");
             conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('" + TENANT_ID + "', 'def', 1, 'Billy Gibbons')");
+            analyzeTable(conn, PARENT_TABLE_NAME);
             conn.close();
             
             conn = nextConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL);
@@ -216,6 +222,7 @@ public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
             conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('AC/DC', 'abc', 1, 'Bon Scott')");
             conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('" + TENANT_ID + "', '" + TENANT_TYPE_ID + "', 1, 'Billy Gibbons')");
             conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('" + TENANT_ID + "', 'def', 1, 'Billy Gibbons')");
+            analyzeTable(conn, PARENT_TABLE_NAME);
             conn.close();
             
             conn = nextConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL);
@@ -253,6 +260,7 @@ public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
             conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME_NO_TENANT_TYPE_ID + " (tenant_id, id, user) values ('AC/DC', 1, 'Bon Scott')");
             conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME_NO_TENANT_TYPE_ID + " (tenant_id, id, user) values ('" + TENANT_ID + "', 1, 'Billy Gibbons')");
             conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME_NO_TENANT_TYPE_ID + " (tenant_id, id, user) values ('" + TENANT_ID + "', 2, 'Billy Gibbons')");
+            analyzeTable(conn, PARENT_TABLE_NAME_NO_TENANT_TYPE_ID);
             conn.close();
             
             conn = nextConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL);
@@ -289,7 +297,7 @@ public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
             conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('AC/DC', 'abc', 1, 'Bon Scott')");
             conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('" + TENANT_ID + "', '" + TENANT_TYPE_ID + "', 1, 'Billy Gibbons')");
             conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('" + TENANT_ID + "', 'def', 1, 'Billy Gibbons')");
-            
+            analyzeTable(conn, PARENT_TABLE_NAME);
             conn.close();
             
             conn = nextConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL);
@@ -320,7 +328,7 @@ public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
             conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME_NO_TENANT_TYPE_ID + " (tenant_id, id, user) values ('AC/DC', 1, 'Bon Scott')");
             conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME_NO_TENANT_TYPE_ID + " (tenant_id, id, user) values ('" + TENANT_ID + "', 1, 'Billy Gibbons')");
             conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME_NO_TENANT_TYPE_ID + " (tenant_id, id, user) values ('" + TENANT_ID + "', 2, 'Billy Gibbons')");
-            
+            analyzeTable(conn, PARENT_TABLE_NAME_NO_TENANT_TYPE_ID);
             conn.close();
             
             conn = nextConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL);
@@ -350,6 +358,7 @@ public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
             conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('AC/DC', 'aaa', 1, 'Bon Scott')");
             conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('" + TENANT_ID + "', '" + TENANT_TYPE_ID + "', 1, 'Billy Gibbons')");
             conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('" + TENANT_ID + "', 'def', 2, 'Billy Gibbons')");
+            analyzeTable(conn, PARENT_TABLE_NAME);
             conn.close();
             
             conn = nextConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL);
@@ -384,6 +393,7 @@ public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
             conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('AC/DC', 'aaa', 1, 'Bon Scott')");
             conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('" + TENANT_ID + "', '" + TENANT_TYPE_ID + "', 1, 'Billy Gibbons')");
             conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('" + TENANT_ID + "', 'def', 2, 'Billy Gibbons')");
+            analyzeTable(conn, PARENT_TABLE_NAME);
             conn.close();
             
             conn = nextConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL);
@@ -432,6 +442,7 @@ public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
             conn = nextConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL);
             conn.setAutoCommit(true);
             conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_type_id, id, user) values ('" + TENANT_TYPE_ID + "', 1, 'Billy Gibbons')");
+            analyzeTable(conn, PARENT_TABLE_NAME);
             conn.close();
 
             conn = nextConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL);
@@ -460,6 +471,10 @@ public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
         }
     }
     
+    private void analyzeTable(Connection conn, String tableName) throws IOException, SQLException {
+        String query = "ANALYZE " + tableName;
+        conn.createStatement().execute(query);
+    }
     @Test
     public void testUpsertValuesUsingViewWithNoWhereClause() throws Exception {
         Connection conn = nextConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java
index 5452d38..20104a4 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java
@@ -122,7 +122,7 @@ public class TenantSpecificViewIndexIT extends BaseTenantSpecificViewIndexIT {
         if(localIndex){
             assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _LOCAL_IDX_MT_BASE ['a',-32768,'f']\nCLIENT MERGE SORT",QueryUtil.getExplainPlan(rs));
         } else {
-            assertEquals("CLIENT PARALLEL 4-WAY RANGE SCAN OVER _IDX_MT_BASE ['a',-32768,'f']",QueryUtil.getExplainPlan(rs));
+            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER _IDX_MT_BASE ['a',-32768,'f']",QueryUtil.getExplainPlan(rs));
         }
         
         try {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexIT.java
index 8aaebba..cdd44f1 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexIT.java
@@ -58,7 +58,6 @@ public class MutableIndexIT extends BaseMutableIndexIT {
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(3);
         // Don't split intra region so we can more easily know that the n-way parallelization is for the explain plan
-        props.put(QueryServices.MAX_INTRA_REGION_PARALLELIZATION_ATTRIB, Integer.toString(1));
         // Forces server cache to be used
         props.put(QueryServices.INDEX_MUTATE_BATCH_SIZE_THRESHOLD_ATTRIB, Integer.toString(2));
         props.put(QueryServices.DROP_METADATA_ATTRIB, Boolean.toString(true));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cdc938e/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/SaltedIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/SaltedIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/SaltedIndexIT.java
index b269cd1..8eae5f9 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/SaltedIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/SaltedIndexIT.java
@@ -53,8 +53,6 @@ public class SaltedIndexIT extends BaseIndexIT {
     @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(3);
-        // Don't split intra region so we can more easily know that the n-way parallelization is for the explain plan
-        props.put(QueryServices.MAX_INTRA_REGION_PARALLELIZATION_ATTRIB, Integer.toString(1));
         // Forces server cache to be used
         props.put(QueryServices.INDEX_MUTATE_BATCH_SIZE_THRESHOLD_ATTRIB, Integer.toString(2));
         // Drop the HBase table metadata for this test
@@ -123,7 +121,9 @@ public class SaltedIndexIT extends BaseIndexIT {
         stmt.setString(2, "y");
         stmt.execute();
         conn.commit();
-        
+        stmt = conn.prepareStatement("ANALYZE "+DATA_TABLE_FULL_NAME);
+        stmt.execute();
+
         query = "SELECT * FROM " + INDEX_TABLE_FULL_NAME;
         rs = conn.createStatement().executeQuery(query);
         assertTrue(rs.next());
@@ -177,10 +177,10 @@ public class SaltedIndexIT extends BaseIndexIT {
         assertFalse(rs.next());
         rs = conn.createStatement().executeQuery("EXPLAIN " + query);
         expectedPlan = tableSaltBuckets == null ? 
-                "CLIENT PARALLEL 1-WAY POINT LOOKUP ON 1 KEY OVER " + DATA_TABLE_FULL_NAME + "\n" +
+                "CLIENT PARALLEL 3-WAY POINT LOOKUP ON 1 KEY OVER " + DATA_TABLE_FULL_NAME + "\n" +
                 "    SERVER SORTED BY [V]\n" + 
                 "CLIENT MERGE SORT" :
-                    "CLIENT PARALLEL 1-WAY POINT LOOKUP ON 1 KEY OVER " + DATA_TABLE_FULL_NAME + "\n" + 
+                    "CLIENT PARALLEL 2-WAY POINT LOOKUP ON 1 KEY OVER " + DATA_TABLE_FULL_NAME + "\n" + 
                     "    SERVER SORTED BY [V]\n" + 
                     "CLIENT MERGE SORT";
         assertEquals(expectedPlan,QueryUtil.getExplainPlan(rs));
@@ -199,15 +199,16 @@ public class SaltedIndexIT extends BaseIndexIT {
         assertFalse(rs.next());
         rs = conn.createStatement().executeQuery("EXPLAIN " + query);
         expectedPlan = tableSaltBuckets == null ? 
-             "CLIENT PARALLEL 1-WAY FULL SCAN OVER " + DATA_TABLE_FULL_NAME + "\n" +
+             "CLIENT PARALLEL 3-WAY FULL SCAN OVER " + DATA_TABLE_FULL_NAME + "\n" +
              "    SERVER FILTER BY V >= 'x'\n" + 
              "    SERVER 2 ROW LIMIT\n" + 
              "CLIENT 2 ROW LIMIT" :
-                 "CLIENT PARALLEL 3-WAY FULL SCAN OVER " + DATA_TABLE_FULL_NAME + "\n" +
+                 "CLIENT PARALLEL 4-WAY FULL SCAN OVER " + DATA_TABLE_FULL_NAME + "\n" +
                  "    SERVER FILTER BY V >= 'x'\n" + 
                  "    SERVER 2 ROW LIMIT\n" + 
                  "CLIENT MERGE SORT\n" + 
                  "CLIENT 2 ROW LIMIT";
-        assertEquals(expectedPlan,QueryUtil.getExplainPlan(rs));
+        String explainPlan = QueryUtil.getExplainPlan(rs);
+        assertEquals(expectedPlan,explainPlan);
     }
 }