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/10/28 23:55:07 UTC

[2/3] git commit: PHOENIX-1391 Remove obsolete hint

PHOENIX-1391 Remove obsolete hint

Conflicts:
	phoenix-core/src/main/java/org/apache/phoenix/execute/BaseQueryPlan.java


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

Branch: refs/heads/3.0
Commit: 4767f139b2ef125549a4a727bdb33bbf9d54bda2
Parents: 6c94dc6
Author: James Taylor <jt...@salesforce.com>
Authored: Tue Oct 28 14:08:08 2014 -0700
Committer: James Taylor <jt...@salesforce.com>
Committed: Tue Oct 28 15:28:14 2014 -0700

----------------------------------------------------------------------
 .../end2end/SkipScanAfterManualSplitIT.java        |  2 +-
 .../phoenix/coprocessor/MetaDataEndpointImpl.java  |  2 +-
 .../phoenix/coprocessor/MetaDataProtocol.java      |  2 +-
 .../org/apache/phoenix/execute/BaseQueryPlan.java  |  6 ++++++
 .../org/apache/phoenix/iterate/ExplainTable.java   | 17 +++++++++++++++--
 .../apache/phoenix/iterate/ParallelIterators.java  |  2 +-
 .../phoenix/jdbc/PhoenixDatabaseMetaData.java      | 15 +++++++--------
 .../java/org/apache/phoenix/parse/HintNode.java    |  9 ++++-----
 .../phoenix/query/ConnectionQueryServices.java     |  2 +-
 .../phoenix/query/ConnectionQueryServicesImpl.java |  4 ++--
 .../query/ConnectionlessQueryServicesImpl.java     |  2 +-
 .../query/DelegateConnectionQueryServices.java     |  4 ++--
 .../org/apache/phoenix/schema/MetaDataClient.java  |  2 +-
 .../java/org/apache/phoenix/util/ScanUtil.java     | 13 +++++++++----
 .../phoenix/query/BaseConnectionlessQueryTest.java |  2 ++
 .../org/apache/phoenix/query/QueryPlanTest.java    |  8 ++++++++
 16 files changed, 62 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/4767f139/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipScanAfterManualSplitIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipScanAfterManualSplitIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipScanAfterManualSplitIT.java
index a07ad0e..3c0344c 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipScanAfterManualSplitIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SkipScanAfterManualSplitIT.java
@@ -128,7 +128,7 @@ public class SkipScanAfterManualSplitIT extends BaseHBaseManagedTimeIT {
             assertEquals(nRegions, nInitialRegions);
             
             int nRows = 2;
-            String query = "SELECT /*+ NO_INTRA_REGION_PARALLELIZATION */ count(*) FROM S WHERE a IN ('tl','jt',' a',' b',' c',' d')";
+            String query = "SELECT count(*) FROM S WHERE a IN ('tl','jt',' a',' b',' c',' d')";
             ResultSet rs1 = conn.createStatement().executeQuery(query);
             assertTrue(rs1.next());
             nRegions = services.getAllTableRegions(TABLE_NAME_BYTES).size();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4767f139/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 38277c5..0327c35 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
@@ -1384,7 +1384,7 @@ public class MetaDataEndpointImpl extends BaseEndpointCoprocessor implements Met
     }
     
     @Override
-    public void incrementTableTimeStamp(byte[] tenantId, byte[] schemaName, byte[] tableName, final long clientTimeStamp)
+    public void clearTableFromCache(byte[] tenantId, byte[] schemaName, byte[] tableName, final long clientTimeStamp)
             throws IOException {
         try {
             byte[] key = SchemaUtil.getTableKey(tenantId, schemaName, tableName);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4767f139/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
index 7979757..76fb129 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
@@ -266,7 +266,7 @@ public interface MetaDataProtocol extends CoprocessorProtocol {
      */
     void clearCache();
     
-    void incrementTableTimeStamp(byte[] tenantId, byte[] schema, byte[] tableName, long clientTimestamp)
+    void clearTableFromCache(byte[] tenantId, byte[] schema, byte[] tableName, long clientTimestamp)
             throws IOException;
     
     /**

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4767f139/phoenix-core/src/main/java/org/apache/phoenix/execute/BaseQueryPlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/BaseQueryPlan.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/BaseQueryPlan.java
index 3a59828..9053bab 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/BaseQueryPlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/BaseQueryPlan.java
@@ -36,6 +36,7 @@ import org.apache.phoenix.iterate.ParallelIterators.ParallelIteratorFactory;
 import org.apache.phoenix.iterate.ResultIterator;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.parse.FilterableStatement;
+import org.apache.phoenix.parse.HintNode.Hint;
 import org.apache.phoenix.schema.TableRef;
 import org.apache.phoenix.util.SQLCloseable;
 import org.apache.phoenix.util.SQLCloseables;
@@ -132,6 +133,11 @@ public abstract class BaseQueryPlan implements QueryPlan {
         }
         
         Scan scan = context.getScan();
+        
+        if (statement.getHint().hasHint(Hint.SMALL)) {
+            ScanUtil.setSmall(scan, true);
+        }
+        
         // Set producer on scan so HBase server does round robin processing
         //setProducer(scan);
         // Set the time range on the scan so we don't get back rows newer than when the statement was compiled

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4767f139/phoenix-core/src/main/java/org/apache/phoenix/iterate/ExplainTable.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/ExplainTable.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/ExplainTable.java
index 40a0cff..8c04383 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/iterate/ExplainTable.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/ExplainTable.java
@@ -31,9 +31,12 @@ import org.apache.hadoop.hbase.filter.PageFilter;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.compile.GroupByCompiler.GroupBy;
+import org.apache.phoenix.compile.OrderByCompiler.OrderBy;
 import org.apache.phoenix.compile.ScanRanges;
 import org.apache.phoenix.compile.StatementContext;
 import org.apache.phoenix.coprocessor.BaseScannerRegionObserver;
+import org.apache.phoenix.parse.HintNode;
+import org.apache.phoenix.parse.HintNode.Hint;
 import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.query.KeyRange.Bound;
 import org.apache.phoenix.schema.PDataType;
@@ -50,15 +53,19 @@ public abstract class ExplainTable {
     protected final StatementContext context;
     protected final TableRef tableRef;
     protected final GroupBy groupBy;
+    protected final OrderBy orderBy;
+    protected final HintNode hint;
    
     public ExplainTable(StatementContext context, TableRef table) {
-        this(context,table,GroupBy.EMPTY_GROUP_BY);
+        this(context,table,GroupBy.EMPTY_GROUP_BY, OrderBy.EMPTY_ORDER_BY, HintNode.EMPTY_HINT_NODE);
     }
 
-    public ExplainTable(StatementContext context, TableRef table, GroupBy groupBy) {
+    public ExplainTable(StatementContext context, TableRef table, GroupBy groupBy, OrderBy orderBy, HintNode hintNode) {
         this.context = context;
         this.tableRef = table;
         this.groupBy = groupBy;
+        this.orderBy = orderBy;
+        this.hint = hintNode;
     }
 
     private boolean explainSkipScan(StringBuilder buf) {
@@ -90,6 +97,12 @@ public abstract class ExplainTable {
         StringBuilder buf = new StringBuilder(prefix);
         ScanRanges scanRanges = context.getScanRanges();
         boolean hasSkipScanFilter = false;
+        if (hint.hasHint(Hint.SMALL)) {
+            buf.append("SMALL ");
+        }
+        if (OrderBy.REV_ROW_KEY_ORDER_BY.equals(orderBy)) {
+            buf.append("REVERSE ");
+        }
         if (scanRanges.isEverything()) {
             buf.append("FULL SCAN ");
         } else {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4767f139/phoenix-core/src/main/java/org/apache/phoenix/iterate/ParallelIterators.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/ParallelIterators.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/ParallelIterators.java
index 43fa4f5..f5c4027 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/iterate/ParallelIterators.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/ParallelIterators.java
@@ -110,7 +110,7 @@ public class ParallelIterators extends ExplainTable implements ResultIterators {
 
     public ParallelIterators(QueryPlan plan, Integer perScanLimit, ParallelIteratorFactory iteratorFactory)
             throws SQLException {
-        super(plan.getContext(), plan.getTableRef(), plan.getGroupBy());
+        super(plan.getContext(), plan.getTableRef(), plan.getGroupBy(), plan.getOrderBy(), plan.getStatement().getHint());
         this.plan = plan;
         StatementContext context = plan.getContext();
         TableRef tableRef = plan.getTableRef();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4767f139/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 5560d48..5f2e8b8 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
@@ -49,7 +49,6 @@ import org.apache.phoenix.hbase.index.util.VersionUtil;
 import org.apache.phoenix.iterate.DelegateResultIterator;
 import org.apache.phoenix.iterate.MaterializedResultIterator;
 import org.apache.phoenix.iterate.ResultIterator;
-import org.apache.phoenix.parse.HintNode.Hint;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.PDataType;
 import org.apache.phoenix.schema.PDatum;
@@ -333,7 +332,7 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData, org.apache.pho
 
     @Override
     public ResultSet getCatalogs() throws SQLException {
-        StringBuilder buf = new StringBuilder("select /*+" + Hint.NO_INTRA_REGION_PARALLELIZATION + "*/\n" +
+        StringBuilder buf = new StringBuilder("select \n" +
                 " DISTINCT " + TENANT_ID + " " + TABLE_CAT +
                 " from " + SYSTEM_CATALOG + " " + SYSTEM_CATALOG_ALIAS +
                 " where " + COLUMN_NAME + " is null" +
@@ -389,7 +388,7 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData, org.apache.pho
     @Override
     public ResultSet getColumns(String catalog, String schemaPattern, String tableNamePattern, String columnNamePattern)
             throws SQLException {
-        StringBuilder buf = new StringBuilder("select /*+" + Hint.NO_INTRA_REGION_PARALLELIZATION + "*/\n " +
+        StringBuilder buf = new StringBuilder("select \n " +
                 TENANT_ID + " " + TABLE_CAT + "," + // use this for tenant id
                 TABLE_SCHEM + "," +
                 TABLE_NAME + " ," +
@@ -647,7 +646,7 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData, org.apache.pho
         if (unique) { // No unique indexes
             return emptyResultSet;
         }
-        StringBuilder buf = new StringBuilder("select /*+" + Hint.NO_INTRA_REGION_PARALLELIZATION + "*/\n" +
+        StringBuilder buf = new StringBuilder("select \n" +
                 TENANT_ID + " " + TABLE_CAT + ",\n" + // use this column for column family name
                 TABLE_SCHEM + ",\n" +
                 DATA_TABLE_NAME + " " + TABLE_NAME + ",\n" +
@@ -800,7 +799,7 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData, org.apache.pho
         if (table == null || table.length() == 0) {
             return emptyResultSet;
         }
-        StringBuilder buf = new StringBuilder("select /*+" + Hint.NO_INTRA_REGION_PARALLELIZATION + "*/\n" +
+        StringBuilder buf = new StringBuilder("select \n" +
                 TENANT_ID + " " + TABLE_CAT + "," + // use catalog for tenant_id
                 TABLE_SCHEM + "," +
                 TABLE_NAME + " ," +
@@ -874,7 +873,7 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData, org.apache.pho
 
     @Override
     public ResultSet getSchemas(String catalog, String schemaPattern) throws SQLException {
-        StringBuilder buf = new StringBuilder("select /*+" + Hint.NO_INTRA_REGION_PARALLELIZATION + "*/\n distinct " +
+        StringBuilder buf = new StringBuilder("select distinct \n" +
                 TENANT_ID + " " + TABLE_CATALOG + "," + // no catalog for tables
                 TABLE_SCHEM +
                 " from " + SYSTEM_CATALOG + " " + SYSTEM_CATALOG_ALIAS +
@@ -899,7 +898,7 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData, org.apache.pho
 
     @Override
     public ResultSet getSuperTables(String catalog, String schemaPattern, String tableNamePattern) throws SQLException {
-        StringBuilder buf = new StringBuilder("select /*+" + Hint.NO_INTRA_REGION_PARALLELIZATION + "*/\n" +
+        StringBuilder buf = new StringBuilder("select \n" +
                 TENANT_ID + " " + TABLE_CAT + "," + // Use tenantId for catalog
                 TABLE_SCHEM + "," +
                 TABLE_NAME + "," +
@@ -986,7 +985,7 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData, org.apache.pho
     @Override
     public ResultSet getTables(String catalog, String schemaPattern, String tableNamePattern, String[] types)
             throws SQLException {
-        StringBuilder buf = new StringBuilder("select /*+" + Hint.NO_INTRA_REGION_PARALLELIZATION + "*/\n" +
+        StringBuilder buf = new StringBuilder("select \n" +
                 TENANT_ID + " " + TABLE_CAT + "," + // tenant_id is the catalog
                 TABLE_SCHEM + "," +
                 TABLE_NAME + " ," +

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4767f139/phoenix-core/src/main/java/org/apache/phoenix/parse/HintNode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/HintNode.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/HintNode.java
index 0ded0b5..ea20114 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/HintNode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/HintNode.java
@@ -55,11 +55,6 @@ public class HintNode {
          */
         SKIP_SCAN_HASH_JOIN,
         /**
-         * Prevents the spawning of multiple threads during
-         * query processing.
-         */
-        NO_INTRA_REGION_PARALLELIZATION,
-        /**
         * Prevents the usage of indexes, forcing usage
         * of the data table for a query.
         */
@@ -100,6 +95,10 @@ public class HintNode {
         * between 2 selected columns this will be give better performance.
         */
       NO_SEEK_TO_COLUMN,
+      /**
+       * Saves an RPC call on the scan. See Scan.setSmall(true) in HBase documentation.
+       */
+     SMALL,
     };
 
     private final Map<Hint,String> hints;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4767f139/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 965a515..da61e51 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
@@ -104,7 +104,7 @@ public interface ConnectionQueryServices extends QueryServices, MetaDataMutated
     public boolean supportsFeature(Feature feature);
     
     public String getUserName();
-    public void incrementTableTimeStamp(final byte[] tenantId, final byte[] schemaName, final byte[] tableName, long clientTS) throws SQLException;
+    public void clearTableFromCache(final byte[] tenantId, final byte[] schemaName, final byte[] tableName, long clientTS) throws SQLException;
 
     public PTableStats getTableStats(byte[] physicalName, long clientTimeStamp) throws SQLException;
     

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4767f139/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 3e46a30..557f6d4 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
@@ -1846,7 +1846,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     }
 
     @Override
-    public void incrementTableTimeStamp(final byte[] tenantId, final byte[] schemaName, final byte[] tableName,
+    public void clearTableFromCache(final byte[] tenantId, final byte[] schemaName, final byte[] tableName,
             final long clientTS) throws SQLException {
         // clear the meta data cache for the table here
         try {
@@ -1858,7 +1858,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         new Batch.Call<MetaDataProtocol, MetaDataMutationResult>() {
                     @Override
                     public MetaDataMutationResult call(MetaDataProtocol instance) throws IOException {
-                        instance.incrementTableTimeStamp(tenantId, schemaName, tableName, clientTS);
+                        instance.clearTableFromCache(tenantId, schemaName, tableName, clientTS);
                         // TODO : Should this really return a result?Return null
                         return new MetaDataMutationResult();
                     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4767f139/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 7d0a109..b29f2b6 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
@@ -229,7 +229,7 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
     }
     
     @Override
-    public void incrementTableTimeStamp(byte[] tenantId, byte[] schemaName, byte[] tableName, long clientTS)
+    public void clearTableFromCache(byte[] tenantId, byte[] schemaName, byte[] tableName, long clientTS)
             throws SQLException {}
     // TODO: share this with ConnectionQueryServicesImpl
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4767f139/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 defad5b..ae0b689 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
@@ -229,9 +229,9 @@ public class DelegateConnectionQueryServices extends DelegateQueryServices imple
     }
     
     @Override
-    public void incrementTableTimeStamp(byte[] tenantId, byte[] schemaName, byte[] tableName, long clientTS)
+    public void clearTableFromCache(byte[] tenantId, byte[] schemaName, byte[] tableName, long clientTS)
             throws SQLException {
-        getDelegate().incrementTableTimeStamp(tenantId, schemaName, tableName, clientTS);
+        getDelegate().clearTableFromCache(tenantId, schemaName, tableName, clientTS);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4767f139/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 7421f86..a460f9f 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
@@ -621,7 +621,7 @@ public class MetaDataClient {
 
         // We need to update the stats table so that client will pull the new one with
         // the updated stats.
-        connection.getQueryServices().incrementTableTimeStamp(tenantIdBytes,
+        connection.getQueryServices().clearTableFromCache(tenantIdBytes,
                 Bytes.toBytes(SchemaUtil.getSchemaNameFromFullName(physicalName.getString())),
                 Bytes.toBytes(SchemaUtil.getTableNameFromFullName(physicalName.getString())), clientTimeStamp);
         return rowCount;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4767f139/phoenix-core/src/main/java/org/apache/phoenix/util/ScanUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/ScanUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/ScanUtil.java
index 90fbba7..bfbaa37 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/ScanUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/ScanUtil.java
@@ -59,7 +59,11 @@ import com.google.common.collect.Lists;
 public class ScanUtil {
     public static final int[] SINGLE_COLUMN_SLOT_SPAN = new int[1];
     private static final byte[] ZERO_BYTE_ARRAY = new byte[1024];
-
+    // Don't use constants in Scan as they didn't exist before.
+    // Doing this maintains backward compatibility.
+    private static final String REVERSED_ATTR = "_reversed_";
+    private static final String SMALL_ATTRIB = "_small_";
+    
     private ScanUtil() {
     }
 
@@ -432,10 +436,7 @@ public class ScanUtil {
         return key;
     }
 
-    private static final String REVERSED_ATTR = "_reversed_";
-    
     public static void setReversed(Scan scan) {
-        // TODO: set attribute dynamically here to prevent dependency on newer HBase release
         scan.setAttribute(REVERSED_ATTR, PDataType.TRUE_BYTES);
     }
 
@@ -538,4 +539,8 @@ public class ScanUtil {
         }
         return tenantId;
     }
+
+    public static void setSmall(Scan scan, boolean b) {
+        scan.setAttribute(SMALL_ATTRIB, Bytes.toBytes(true));
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4767f139/phoenix-core/src/test/java/org/apache/phoenix/query/BaseConnectionlessQueryTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseConnectionlessQueryTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseConnectionlessQueryTest.java
index 8ac322f..8f17a7c 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseConnectionlessQueryTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseConnectionlessQueryTest.java
@@ -27,6 +27,7 @@ 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.MULTI_CF_NAME;
 import static org.apache.phoenix.util.TestUtil.PHOENIX_CONNECTIONLESS_JDBC_URL;
+import static org.apache.phoenix.util.TestUtil.PTSDB2_NAME;
 import static org.apache.phoenix.util.TestUtil.PTSDB3_NAME;
 import static org.apache.phoenix.util.TestUtil.PTSDB_NAME;
 import static org.apache.phoenix.util.TestUtil.TABLE_WITH_ARRAY;
@@ -104,6 +105,7 @@ public class BaseConnectionlessQueryTest extends BaseTest {
         ensureTableCreated(getUrl(), ENTITY_HISTORY_TABLE_NAME);
         ensureTableCreated(getUrl(), FUNKY_NAME);
         ensureTableCreated(getUrl(), PTSDB_NAME);
+        ensureTableCreated(getUrl(), PTSDB2_NAME);
         ensureTableCreated(getUrl(), PTSDB3_NAME);
         ensureTableCreated(getUrl(), MULTI_CF_NAME);
         ensureTableCreated(getUrl(), JOIN_ORDER_TABLE_FULL_NAME);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4767f139/phoenix-core/src/test/java/org/apache/phoenix/query/QueryPlanTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/QueryPlanTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/QueryPlanTest.java
index fd22e47..1e3df0b 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/QueryPlanTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/QueryPlanTest.java
@@ -41,6 +41,14 @@ public class QueryPlanTest extends BaseConnectionlessQueryTest {
                 "CLIENT PARALLEL 1-WAY SKIP SCAN ON 3 KEYS OVER PTSDB3 [~'na3'] - [~'na1']\n" + 
                 "    SERVER FILTER BY FIRST KEY ONLY",
 
+                "SELECT /*+ SMALL*/ host FROM PTSDB3 WHERE host IN ('na1', 'na2','na3')",
+                "CLIENT PARALLEL 1-WAY SMALL SKIP SCAN ON 3 KEYS OVER PTSDB3 [~'na3'] - [~'na1']\n" + 
+                "    SERVER FILTER BY FIRST KEY ONLY",
+
+                "SELECT inst,date FROM PTSDB2 WHERE inst = 'na1' ORDER BY inst DESC, date DESC",
+                "CLIENT PARALLEL 1-WAY REVERSE RANGE SCAN OVER PTSDB2 ['na1']\n" +
+                "    SERVER FILTER BY FIRST KEY ONLY",
+
                 "SELECT host FROM PTSDB WHERE inst IS NULL AND host IS NOT NULL AND date >= to_date('2013-01-01')",
                 "CLIENT PARALLEL 1-WAY RANGE SCAN OVER PTSDB [null,not null]\n" + 
                 "    SERVER FILTER BY FIRST KEY ONLY AND DATE >= '2013-01-01 00:00:00.000'",