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/11/07 04:45:41 UTC

[1/4] phoenix git commit: PHOENIX-1416 Given a schema name, DatabaseMetadata.getTables and getColumns calls erroneously match tables without schema

Repository: phoenix
Updated Branches:
  refs/heads/4.0 d0c9dac00 -> 4ed86c0cc


PHOENIX-1416 Given a schema name, DatabaseMetadata.getTables and getColumns calls erroneously match tables without schema


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

Branch: refs/heads/4.0
Commit: 798531c45e88c92b6851de11ca0f3dd8af00e5f0
Parents: d0c9dac
Author: James Taylor <jt...@salesforce.com>
Authored: Thu Nov 6 17:57:57 2014 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Thu Nov 6 17:57:57 2014 -0800

----------------------------------------------------------------------
 .../end2end/QueryDatabaseMetaDataIT.java        | 43 ++++++++++++++++++
 .../org/apache/phoenix/compile/ScanRanges.java  |  8 +++-
 .../phoenix/expression/AndExpression.java       |  4 +-
 .../phoenix/expression/AndOrExpression.java     | 11 +----
 .../apache/phoenix/expression/OrExpression.java |  4 +-
 .../phoenix/compile/WhereOptimizerTest.java     | 47 ++++++++++++++++++++
 6 files changed, 102 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/798531c4/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 f243562..4c5c828 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
@@ -1063,4 +1063,47 @@ public class QueryDatabaseMetaDataIT extends BaseClientManagedTimeIT {
         }
         conn5.close();
     }
+    
+
+    @Test
+    public void testTableWithScemaMetadataScan() throws SQLException {
+        long ts = nextTimestamp();
+        Properties props = new Properties();
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts));
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        
+        conn.createStatement().execute("create table foo.bar(k varchar primary key)");
+        conn.createStatement().execute("create table bar(k varchar primary key)");       
+        conn.close();
+        
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 10));
+        conn = DriverManager.getConnection(getUrl(), props);
+        
+        DatabaseMetaData metaData = conn.getMetaData();
+        ResultSet rs;
+        
+        // Tricky case that requires returning false for null AND true expression
+        rs = metaData.getTables(null, "FOO", "BAR", null);
+        assertTrue(rs.next());
+        assertEquals("FOO",rs.getString("TABLE_SCHEM"));
+        assertEquals("BAR", rs.getString("TABLE_NAME"));
+        assertFalse(rs.next());
+
+        // Tricky case that requires end key to maintain trailing nulls
+        rs = metaData.getTables("", "FOO", "BAR", null);
+        assertTrue(rs.next());
+        assertEquals("FOO",rs.getString("TABLE_SCHEM"));
+        assertEquals("BAR", rs.getString("TABLE_NAME"));
+        assertFalse(rs.next());
+
+        rs = metaData.getTables("", null, "BAR", null);
+        assertTrue(rs.next());
+        assertEquals(null,rs.getString("TABLE_SCHEM"));
+        assertEquals("BAR", rs.getString("TABLE_NAME"));
+        assertTrue(rs.next());
+        assertEquals("FOO",rs.getString("TABLE_SCHEM"));
+        assertEquals("BAR", rs.getString("TABLE_NAME"));
+        assertFalse(rs.next());
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/798531c4/phoenix-core/src/main/java/org/apache/phoenix/compile/ScanRanges.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/ScanRanges.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/ScanRanges.java
index 0ab6368..0842f6a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/ScanRanges.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/ScanRanges.java
@@ -440,9 +440,13 @@ public class ScanRanges {
         if (ScanUtil.getTotalSpan(ranges, slotSpan) < schema.getMaxFields()) {
             return false;
         }
-        for (List<KeyRange> orRanges : ranges) {
+        int lastIndex = ranges.size()-1;
+        for (int i = lastIndex; i >= 0; i--) {
+            List<KeyRange> orRanges = ranges.get(i);
             for (KeyRange keyRange : orRanges) {
-                if (!keyRange.isSingleKey()) {
+                // Special case for single trailing IS NULL. We cannot consider this as a point key because
+                // we strip trailing nulls when we form the key.
+                if (!keyRange.isSingleKey() || (i == lastIndex && keyRange == KeyRange.IS_NULL_RANGE)) {
                     return false;
                 }
             }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/798531c4/phoenix-core/src/main/java/org/apache/phoenix/expression/AndExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/AndExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/AndExpression.java
index e9c2740..bb2dc7e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/AndExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/AndExpression.java
@@ -79,8 +79,8 @@ public class AndExpression extends AndOrExpression {
     }
 
     @Override
-    protected boolean getStopValue() {
-        return Boolean.FALSE;
+    protected boolean isStopValue(Boolean value) {
+        return !Boolean.TRUE.equals(value);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/798531c4/phoenix-core/src/main/java/org/apache/phoenix/expression/AndOrExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/AndOrExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/AndOrExpression.java
index eebcd34..89ad02e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/AndOrExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/AndOrExpression.java
@@ -21,7 +21,6 @@ import java.util.BitSet;
 import java.util.List;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-
 import org.apache.phoenix.schema.PDataType;
 import org.apache.phoenix.schema.tuple.Tuple;
 
@@ -45,11 +44,6 @@ public abstract class AndOrExpression extends BaseCompoundExpression {
     }
     
     @Override
-    public int hashCode() {
-        return 31 * super.hashCode() + Boolean.valueOf(this.getStopValue()).hashCode();
-    }
-
-    @Override
     public PDataType getDataType() {
         return PDataType.BOOLEAN;
     }
@@ -67,7 +61,6 @@ public abstract class AndOrExpression extends BaseCompoundExpression {
     @Override
     public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
         boolean isNull = false;
-        boolean stopValue = getStopValue();
         for (int i = 0; i < children.size(); i++) {
             Expression child = children.get(i);
             // If partial state is available, then use that to know we've already evaluated this
@@ -77,7 +70,7 @@ public abstract class AndOrExpression extends BaseCompoundExpression {
                 // evaluate versus getValue code path.
                 if (child.evaluate(tuple, ptr)) {
                     // Short circuit if we see our stop value
-                    if (Boolean.valueOf(stopValue).equals(PDataType.BOOLEAN.toObject(ptr, child.getDataType()))) {
+                    if (isStopValue((Boolean)PDataType.BOOLEAN.toObject(ptr, child.getDataType()))) {
                         return true;
                     } else if (partialEvalState != null) {
                         partialEvalState.set(i);
@@ -93,5 +86,5 @@ public abstract class AndOrExpression extends BaseCompoundExpression {
         return true;
     }
 
-    protected abstract boolean getStopValue();
+    protected abstract boolean isStopValue(Boolean value);
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/798531c4/phoenix-core/src/main/java/org/apache/phoenix/expression/OrExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/OrExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/OrExpression.java
index e8565c5..5b1b62e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/OrExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/OrExpression.java
@@ -38,8 +38,8 @@ public class OrExpression extends AndOrExpression {
     }
 
     @Override
-    protected boolean getStopValue() {
-        return Boolean.TRUE;
+    protected boolean isStopValue(Boolean value) {
+        return Boolean.TRUE.equals(value);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/798531c4/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereOptimizerTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereOptimizerTest.java b/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereOptimizerTest.java
index 032768b..1ce6c02 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereOptimizerTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereOptimizerTest.java
@@ -45,6 +45,7 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.filter.RowKeyComparisonFilter;
 import org.apache.phoenix.filter.SingleKeyValueComparisonFilter;
 import org.apache.phoenix.filter.SkipScanFilter;
@@ -1734,4 +1735,50 @@ public class WhereOptimizerTest extends BaseConnectionlessQueryTest {
         return conn;
     }
     
+    @Test
+    public void testTrailingIsNull() throws Exception {
+        String baseTableDDL = "CREATE TABLE t(\n " + 
+                "  a VARCHAR,\n" + 
+                "  b VARCHAR,\n" + 
+                "  CONSTRAINT pk PRIMARY KEY (a, b))";
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute(baseTableDDL);
+        conn.close();
+        
+        String query = "SELECT * FROM t WHERE a = 'a' and b is null";
+        StatementContext context = compileStatement(query, Collections.<Object>emptyList());
+        Scan scan = context.getScan();
+        Filter filter = scan.getFilter();
+        assertNull(filter);
+        assertArrayEquals(Bytes.toBytes("a"), scan.getStartRow());
+        assertArrayEquals(ByteUtil.concat(Bytes.toBytes("a"), QueryConstants.SEPARATOR_BYTE_ARRAY, QueryConstants.SEPARATOR_BYTE_ARRAY), scan.getStopRow());
+    }
+    
+    
+    @Test
+    public void testTrailingIsNullWithOr() throws Exception {
+        String baseTableDDL = "CREATE TABLE t(\n " + 
+                "  a VARCHAR,\n" + 
+                "  b VARCHAR,\n" + 
+                "  CONSTRAINT pk PRIMARY KEY (a, b))";
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute(baseTableDDL);
+        conn.close();
+        
+        String query = "SELECT * FROM t WHERE a = 'a' and (b is null or b = 'b')";
+        StatementContext context = compileStatement(query, Collections.<Object>emptyList());
+        Scan scan = context.getScan();
+        Filter filter = scan.getFilter();
+        assertTrue(filter instanceof SkipScanFilter);
+        SkipScanFilter skipScan = (SkipScanFilter)filter;
+        List<List<KeyRange>>slots = skipScan.getSlots();
+        assertEquals(2,slots.size());
+        assertEquals(1,slots.get(0).size());
+        assertEquals(2,slots.get(1).size());
+        assertEquals(KeyRange.getKeyRange(Bytes.toBytes("a")), slots.get(0).get(0));
+        assertTrue(KeyRange.IS_NULL_RANGE == slots.get(1).get(0));
+        assertEquals(KeyRange.getKeyRange(Bytes.toBytes("b")), slots.get(1).get(1));
+        assertArrayEquals(Bytes.toBytes("a"), scan.getStartRow());
+        assertArrayEquals(ByteUtil.concat(Bytes.toBytes("a"), QueryConstants.SEPARATOR_BYTE_ARRAY, Bytes.toBytes("b"), QueryConstants.SEPARATOR_BYTE_ARRAY), scan.getStopRow());
+    }
 }


[2/4] phoenix git commit: PHOENIX-1413 Add Phoenix coprocessors with configurable priority

Posted by ja...@apache.org.
PHOENIX-1413 Add Phoenix coprocessors with configurable priority


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

Branch: refs/heads/4.0
Commit: b46d3e23f7fdd357a5d8845559f700a68a879c16
Parents: 798531c
Author: James Taylor <jt...@salesforce.com>
Authored: Thu Nov 6 18:24:54 2014 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Thu Nov 6 18:24:54 2014 -0800

----------------------------------------------------------------------
 .../EndToEndCoveredColumnsIndexBuilderIT.java   |  3 ++-
 .../org/apache/phoenix/hbase/index/Indexer.java | 10 ++++-----
 .../CoveredColumnIndexSpecifierBuilder.java     |  4 ++--
 .../query/ConnectionQueryServicesImpl.java      | 23 ++++++++++----------
 .../org/apache/phoenix/query/QueryServices.java |  2 ++
 .../phoenix/query/QueryServicesOptions.java     |  5 +++++
 6 files changed, 28 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/b46d3e23/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/covered/EndToEndCoveredColumnsIndexBuilderIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/covered/EndToEndCoveredColumnsIndexBuilderIT.java b/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/covered/EndToEndCoveredColumnsIndexBuilderIT.java
index d93b011..9aae820 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/covered/EndToEndCoveredColumnsIndexBuilderIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/covered/EndToEndCoveredColumnsIndexBuilderIT.java
@@ -33,6 +33,7 @@ import java.util.Queue;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
@@ -306,7 +307,7 @@ public class EndToEndCoveredColumnsIndexBuilderIT {
     // initializer blows up.
     indexerOpts.put(CoveredColumnsIndexBuilder.CODEC_CLASS_NAME_KEY,
       CoveredIndexCodecForTesting.class.getName());
-    Indexer.enableIndexing(desc, CoveredColumnsIndexBuilder.class, indexerOpts);
+    Indexer.enableIndexing(desc, CoveredColumnsIndexBuilder.class, indexerOpts, Coprocessor.PRIORITY_USER);
 
     // create the table
     HBaseAdmin admin = UTIL.getHBaseAdmin();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b46d3e23/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/Indexer.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/Indexer.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/Indexer.java
index 9c48a8d..b841410 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/Indexer.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/Indexer.java
@@ -31,7 +31,6 @@ 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.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
@@ -607,18 +606,19 @@ public class Indexer extends BaseRegionObserver {
   /**
    * Enable indexing on the given table
    * @param desc {@link HTableDescriptor} for the table on which indexing should be enabled
-   * @param builder class to use when building the index for this table
-   * @param properties map of custom configuration options to make available to your
+ * @param builder class to use when building the index for this table
+ * @param properties map of custom configuration options to make available to your
    *          {@link IndexBuilder} on the server-side
+ * @param priority TODO
    * @throws IOException the Indexer coprocessor cannot be added
    */
   public static void enableIndexing(HTableDescriptor desc, Class<? extends IndexBuilder> builder,
-      Map<String, String> properties) throws IOException {
+      Map<String, String> properties, int priority) throws IOException {
     if (properties == null) {
       properties = new HashMap<String, String>();
     }
     properties.put(Indexer.INDEX_BUILDER_CONF_KEY, builder.getName());
-    desc.addCoprocessor(Indexer.class.getName(), null, Coprocessor.PRIORITY_USER, properties);
+    desc.addCoprocessor(Indexer.class.getName(), null, priority, properties);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b46d3e23/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/example/CoveredColumnIndexSpecifierBuilder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/example/CoveredColumnIndexSpecifierBuilder.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/example/CoveredColumnIndexSpecifierBuilder.java
index 9fcd5f3..6ac89d1 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/example/CoveredColumnIndexSpecifierBuilder.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/example/CoveredColumnIndexSpecifierBuilder.java
@@ -26,8 +26,8 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.HTableDescriptor;
-
 import org.apache.phoenix.hbase.index.Indexer;
 import org.apache.phoenix.hbase.index.covered.CoveredColumnsIndexBuilder;
 import org.apache.phoenix.hbase.index.covered.IndexCodec;
@@ -137,7 +137,7 @@ public class CoveredColumnIndexSpecifierBuilder {
     // add the codec for the index to the map of options
     Map<String, String> opts = this.convertToMap();
     opts.put(CoveredColumnsIndexBuilder.CODEC_CLASS_NAME_KEY, clazz.getName());
-    Indexer.enableIndexing(desc, CoveredColumnIndexer.class, opts);
+    Indexer.enableIndexing(desc, CoveredColumnIndexer.class, opts, Coprocessor.PRIORITY_USER);
   }
 
   static List<ColumnGroup> getColumns(Configuration conf) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b46d3e23/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 4aa1bac..b73e2dc 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
@@ -604,18 +604,19 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
 
     private void addCoprocessors(byte[] tableName, HTableDescriptor descriptor, PTableType tableType) throws SQLException {
         // The phoenix jar must be available on HBase classpath
+        int priority = props.getInt(QueryServices.COPROCESSOR_PRIORITY_ATTRIB, QueryServicesOptions.DEFAULT_COPROCESSOR_PRIORITY);
         try {
             if (!descriptor.hasCoprocessor(ScanRegionObserver.class.getName())) {
-                descriptor.addCoprocessor(ScanRegionObserver.class.getName(), null, 1, null);
+                descriptor.addCoprocessor(ScanRegionObserver.class.getName(), null, priority, null);
             }
             if (!descriptor.hasCoprocessor(UngroupedAggregateRegionObserver.class.getName())) {
-                descriptor.addCoprocessor(UngroupedAggregateRegionObserver.class.getName(), null, 1, null);
+                descriptor.addCoprocessor(UngroupedAggregateRegionObserver.class.getName(), null, priority, null);
             }
             if (!descriptor.hasCoprocessor(GroupedAggregateRegionObserver.class.getName())) {
-                descriptor.addCoprocessor(GroupedAggregateRegionObserver.class.getName(), null, 1, null);
+                descriptor.addCoprocessor(GroupedAggregateRegionObserver.class.getName(), null, priority, null);
             }
             if (!descriptor.hasCoprocessor(ServerCachingEndpointImpl.class.getName())) {
-                descriptor.addCoprocessor(ServerCachingEndpointImpl.class.getName(), null, 1, null);
+                descriptor.addCoprocessor(ServerCachingEndpointImpl.class.getName(), null, priority, null);
             }
             // TODO: better encapsulation for this
             // Since indexes can't have indexes, don't install our indexing coprocessor for indexes.
@@ -627,11 +628,11 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     && !descriptor.hasCoprocessor(Indexer.class.getName())) {
                 Map<String, String> opts = Maps.newHashMapWithExpectedSize(1);
                 opts.put(CoveredColumnsIndexBuilder.CODEC_CLASS_NAME_KEY, PhoenixIndexCodec.class.getName());
-                Indexer.enableIndexing(descriptor, PhoenixIndexBuilder.class, opts);
+                Indexer.enableIndexing(descriptor, PhoenixIndexBuilder.class, opts, priority);
             }
             if (SchemaUtil.isStatsTable(tableName) && !descriptor.hasCoprocessor(MultiRowMutationEndpoint.class.getName())) {
                 descriptor.addCoprocessor(MultiRowMutationEndpoint.class.getName(),
-                        null, 1, null);
+                        null, priority, null);
             }
             
             if (descriptor.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES) != null
@@ -639,13 +640,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                             .getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES)))) {
                 if (!descriptor.hasCoprocessor(IndexHalfStoreFileReaderGenerator.class.getName())) {
                     descriptor.addCoprocessor(IndexHalfStoreFileReaderGenerator.class.getName(),
-                        null, 1, null);
+                        null, priority, null);
                 }
             } else {
                 if (!descriptor.hasCoprocessor(LocalIndexSplitter.class.getName())
                         && !SchemaUtil.isMetaTable(tableName)
                         && !SchemaUtil.isSequenceTable(tableName)) {
-                    descriptor.addCoprocessor(LocalIndexSplitter.class.getName(), null, 1, null);
+                    descriptor.addCoprocessor(LocalIndexSplitter.class.getName(), null, priority, null);
                 }
             }
 
@@ -653,14 +654,14 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             // stay on the same region.
             if (SchemaUtil.isMetaTable(tableName)) {
                 if (!descriptor.hasCoprocessor(MetaDataEndpointImpl.class.getName())) {
-                    descriptor.addCoprocessor(MetaDataEndpointImpl.class.getName(), null, 1, null);
+                    descriptor.addCoprocessor(MetaDataEndpointImpl.class.getName(), null, priority, null);
                 }
                 if (!descriptor.hasCoprocessor(MetaDataRegionObserver.class.getName())) {
-                    descriptor.addCoprocessor(MetaDataRegionObserver.class.getName(), null, 2, null);
+                    descriptor.addCoprocessor(MetaDataRegionObserver.class.getName(), null, priority + 1, null);
                 }
             } else if (SchemaUtil.isSequenceTable(tableName)) {
                 if (!descriptor.hasCoprocessor(SequenceRegionObserver.class.getName())) {
-                    descriptor.addCoprocessor(SequenceRegionObserver.class.getName(), null, 1, null);
+                    descriptor.addCoprocessor(SequenceRegionObserver.class.getName(), null, priority, null);
                 }
             }
         } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b46d3e23/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 72002ae..414ed57 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
@@ -139,6 +139,8 @@ public interface QueryServices extends SQLCloseable {
     public static final String STATS_USE_CURRENT_TIME_ATTRIB = "phoenix.stats.useCurrentTime";
 
     public static final String SEQUENCE_SALT_BUCKETS_ATTRIB = "phoenix.sequence.saltBuckets";
+    public static final String COPROCESSOR_PRIORITY_ATTRIB = "phoenix.coprocessor.priority";
+    
     /**
      * Get executor service used for parallel scans
      */

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b46d3e23/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 7c8ecd4..8491783 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
@@ -60,6 +60,7 @@ import static org.apache.phoenix.query.QueryServices.USE_INDEXES_ATTRIB;
 import java.util.Map.Entry;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec;
 import org.apache.phoenix.schema.SaltingUtil;
 import org.apache.phoenix.trace.util.Tracing;
@@ -156,6 +157,10 @@ public class QueryServicesOptions {
      * Use only first time SYSTEM.SEQUENCE table is created.
      */
     public static final int DEFAULT_SEQUENCE_TABLE_SALT_BUCKETS = SaltingUtil.MAX_BUCKET_NUM;
+    /**
+     * Default value for coprocessor priority is between SYSTEM and USER priority.
+     */
+    public static final int DEFAULT_COPROCESSOR_PRIORITY = Coprocessor.PRIORITY_SYSTEM/2 + Coprocessor.PRIORITY_USER/2; // Divide individually to prevent any overflow
 
     private final Configuration config;
 


[3/4] phoenix git commit: PHOENIX-1394 getColumnDisplaySize() method returns incorrect value for varchar columns (Samarth Jain)

Posted by ja...@apache.org.
PHOENIX-1394 getColumnDisplaySize() method returns incorrect value for varchar columns (Samarth Jain)


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

Branch: refs/heads/4.0
Commit: 7f7425e462536d16eacd271315054f4f989adca1
Parents: b46d3e2
Author: James Taylor <jt...@salesforce.com>
Authored: Thu Nov 6 19:22:08 2014 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Thu Nov 6 19:22:08 2014 -0800

----------------------------------------------------------------------
 .../phoenix/jdbc/PhoenixResultSetMetaData.java  | 18 +++-----
 .../jdbc/PhoenixResultSetMetadataTest.java      | 45 ++++++++++++++++++++
 2 files changed, 52 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/7f7425e4/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixResultSetMetaData.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixResultSetMetaData.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixResultSetMetaData.java
index b58d5ad..30e9862 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixResultSetMetaData.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixResultSetMetaData.java
@@ -55,9 +55,7 @@ import org.apache.phoenix.schema.PDataType;
  * @since 0.1
  */
 public class PhoenixResultSetMetaData implements ResultSetMetaData {
-    private static final int MIN_DISPLAY_WIDTH = 3;
-    private static final int MAX_DISPLAY_WIDTH = 40;
-    private static final int DEFAULT_DISPLAY_WIDTH = 10;
+    static final int DEFAULT_DISPLAY_WIDTH = 40;
     private final RowProjector rowProjector;
     private final PhoenixConnection connection;
     
@@ -85,21 +83,19 @@ public class PhoenixResultSetMetaData implements ResultSetMetaData {
     @Override
     public int getColumnDisplaySize(int column) throws SQLException {
         ColumnProjector projector = rowProjector.getColumnProjector(column-1);
-        int displaySize = Math.max(projector.getName().length(),MIN_DISPLAY_WIDTH);
         PDataType type = projector.getExpression().getDataType();
         if (type == null) {
-            return Math.min(Math.max(displaySize, QueryConstants.NULL_DISPLAY_TEXT.length()), MAX_DISPLAY_WIDTH);
+            return QueryConstants.NULL_DISPLAY_TEXT.length();
         }
         if (type.isCoercibleTo(PDataType.DATE)) {
-            return Math.min(Math.max(displaySize, connection.getDatePattern().length()), MAX_DISPLAY_WIDTH);
+            return connection.getDatePattern().length();
         }
-        if (type.isFixedWidth() && projector.getExpression().getMaxLength() != null) {
-            return Math.min(Math.max(displaySize, projector.getExpression().getMaxLength()), MAX_DISPLAY_WIDTH);
+        if (projector.getExpression().getMaxLength() != null) {
+            return projector.getExpression().getMaxLength();
         }
-        
-        return Math.min(Math.max(displaySize, DEFAULT_DISPLAY_WIDTH), MAX_DISPLAY_WIDTH);
+        return DEFAULT_DISPLAY_WIDTH;
     }
-
+    
     @Override
     public String getColumnLabel(int column) throws SQLException {
         return rowProjector.getColumnProjector(column-1).getName();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7f7425e4/phoenix-core/src/test/java/org/apache/phoenix/jdbc/PhoenixResultSetMetadataTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/jdbc/PhoenixResultSetMetadataTest.java b/phoenix-core/src/test/java/org/apache/phoenix/jdbc/PhoenixResultSetMetadataTest.java
new file mode 100644
index 0000000..9153595
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/jdbc/PhoenixResultSetMetadataTest.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.jdbc;
+
+import static org.junit.Assert.assertEquals;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+
+import org.apache.phoenix.query.BaseConnectionlessQueryTest;
+import org.apache.phoenix.query.QueryConstants;
+import org.junit.Test;
+
+public class PhoenixResultSetMetadataTest extends BaseConnectionlessQueryTest {
+    
+    @Test
+    public void testColumnDisplaySize() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute(
+                "CREATE TABLE T (pk1 CHAR(15) not null, pk2 VARCHAR not null,  v1 VARCHAR(15), v2 DATE, v3 VARCHAR " +
+                "CONSTRAINT pk PRIMARY KEY (pk1, pk2)) ");
+        ResultSet rs = conn.createStatement().executeQuery("SELECT pk1, pk2, v1, v2, NULL FROM T");
+        assertEquals(15, rs.getMetaData().getColumnDisplaySize(1));
+        assertEquals(PhoenixResultSetMetaData.DEFAULT_DISPLAY_WIDTH, rs.getMetaData().getColumnDisplaySize(2));
+        assertEquals(15, rs.getMetaData().getColumnDisplaySize(3));
+        assertEquals(conn.unwrap(PhoenixConnection.class).getDatePattern().length(), rs.getMetaData().getColumnDisplaySize(4));
+        assertEquals(QueryConstants.NULL_DISPLAY_TEXT.length(), rs.getMetaData().getColumnDisplaySize(5));
+    }
+}


[4/4] phoenix git commit: PHOENIX-1418 Optionally display number of parallel chunks in explain plan

Posted by ja...@apache.org.
PHOENIX-1418 Optionally display number of parallel chunks in explain plan


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

Branch: refs/heads/4.0
Commit: 4ed86c0cc78f316256eb119fcc5f8ffe6ed97362
Parents: 7f7425e
Author: James Taylor <jt...@salesforce.com>
Authored: Thu Nov 6 19:42:19 2014 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Thu Nov 6 19:42:19 2014 -0800

----------------------------------------------------------------------
 .../org/apache/phoenix/end2end/StatsCollectorIT.java     | 11 ++++++++---
 .../org/apache/phoenix/iterate/ParallelIterators.java    |  6 +++++-
 .../java/org/apache/phoenix/query/QueryServices.java     |  1 +
 .../org/apache/phoenix/query/QueryServicesOptions.java   |  7 +++++++
 .../org/apache/phoenix/query/QueryServicesTestImpl.java  |  2 ++
 5 files changed, 23 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/4ed86c0c/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
index e7aada6..faa54ea 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java
@@ -43,6 +43,7 @@ import org.apache.phoenix.query.ConnectionQueryServices;
 import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.TestUtil;
 import org.junit.BeforeClass;
@@ -61,6 +62,7 @@ public class StatsCollectorIT extends BaseOwnClusterHBaseManagedTimeIT {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(3);
         // Must update config before starting server
         props.put(QueryServices.STATS_GUIDEPOST_WIDTH_BYTES_ATTRIB, Long.toString(20));
+        props.put(QueryServices.EXPLAIN_CHUNK_COUNT_ATTRIB, Boolean.TRUE.toString());
         setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
     }
 
@@ -298,7 +300,7 @@ public class StatsCollectorIT extends BaseOwnClusterHBaseManagedTimeIT {
             } while (nRegions == nRegionsNow && nTries < 10);
             // FIXME: I see the commit of the stats finishing before this with a lower timestamp that the scan timestamp,
             // yet without this sleep, the query finds the old data. Seems like an HBase bug and a potentially serious one.
-            Thread.sleep(2000);
+            Thread.sleep(3000);
         } finally {
             admin.close();
         }
@@ -320,15 +322,18 @@ public class StatsCollectorIT extends BaseOwnClusterHBaseManagedTimeIT {
         }
         conn.commit();
         
+        ResultSet rs;
         TestUtil.analyzeTable(conn, STATS_TEST_TABLE_NAME);
         List<KeyRange>keyRanges = getAllSplits(conn, STATS_TEST_TABLE_NAME);
         assertEquals(nRows+1, keyRanges.size());
-        
+        rs = conn.createStatement().executeQuery("EXPLAIN SELECT * FROM " + STATS_TEST_TABLE_NAME);
+        assertEquals("CLIENT " + (nRows+1) + "-CHUNK " + "PARALLEL 1-WAY FULL SCAN OVER " + STATS_TEST_TABLE_NAME, QueryUtil.getExplainPlan(rs));
+
         ConnectionQueryServices services = conn.unwrap(PhoenixConnection.class).getQueryServices();
         List<HRegionLocation> regions = services.getAllTableRegions(STATS_TEST_TABLE_BYTES);
         assertEquals(1, regions.size());
  
-        ResultSet rs = conn.createStatement().executeQuery("SELECT GUIDE_POSTS_COUNT, REGION_NAME FROM SYSTEM.STATS WHERE PHYSICAL_NAME='"+STATS_TEST_TABLE_NAME+"' AND REGION_NAME IS NOT NULL");
+        rs = conn.createStatement().executeQuery("SELECT GUIDE_POSTS_COUNT, REGION_NAME FROM SYSTEM.STATS WHERE PHYSICAL_NAME='"+STATS_TEST_TABLE_NAME+"' AND REGION_NAME IS NOT NULL");
         assertTrue(rs.next());
         assertEquals(nRows, rs.getLong(1));
         assertEquals(regions.get(0).getRegionInfo().getRegionNameAsString(), rs.getString(2));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4ed86c0c/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 00458f3..7905d34 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
@@ -56,6 +56,7 @@ import org.apache.phoenix.query.ConnectionQueryServices;
 import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.schema.MetaDataClient;
 import org.apache.phoenix.schema.PColumnFamily;
 import org.apache.phoenix.schema.PTable;
@@ -654,8 +655,11 @@ public class ParallelIterators extends ExplainTable implements ResultIterators {
 
     @Override
     public void explain(List<String> planSteps) {
+        boolean displayChunkCount = context.getConnection().getQueryServices().getProps().getBoolean(
+                QueryServices.EXPLAIN_CHUNK_COUNT_ATTRIB,
+                QueryServicesOptions.DEFAULT_EXPLAIN_CHUNK_COUNT);
         StringBuilder buf = new StringBuilder();
-        buf.append("CLIENT PARALLEL " + size() + "-WAY ");
+        buf.append("CLIENT " + (displayChunkCount ? (this.splits.size() + "-CHUNK ") : "") + "PARALLEL " + size() + "-WAY ");
         explain(buf.toString(),planSteps);
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4ed86c0c/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 414ed57..b074fb4 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
@@ -140,6 +140,7 @@ public interface QueryServices extends SQLCloseable {
 
     public static final String SEQUENCE_SALT_BUCKETS_ATTRIB = "phoenix.sequence.saltBuckets";
     public static final String COPROCESSOR_PRIORITY_ATTRIB = "phoenix.coprocessor.priority";
+    public static final String EXPLAIN_CHUNK_COUNT_ATTRIB = "phoenix.explain.displayChunkCount";
     
     /**
      * Get executor service used for parallel scans

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4ed86c0c/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 8491783..4894b18 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
@@ -21,6 +21,7 @@ import static org.apache.phoenix.query.QueryServices.CALL_QUEUE_PRODUCER_ATTRIB_
 import static org.apache.phoenix.query.QueryServices.CALL_QUEUE_ROUND_ROBIN_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.DATE_FORMAT_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.DROP_METADATA_ATTRIB;
+import static org.apache.phoenix.query.QueryServices.EXPLAIN_CHUNK_COUNT_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;
@@ -161,6 +162,7 @@ public class QueryServicesOptions {
      * Default value for coprocessor priority is between SYSTEM and USER priority.
      */
     public static final int DEFAULT_COPROCESSOR_PRIORITY = Coprocessor.PRIORITY_SYSTEM/2 + Coprocessor.PRIORITY_USER/2; // Divide individually to prevent any overflow
+    public static final boolean DEFAULT_EXPLAIN_CHUNK_COUNT = true;
 
     private final Configuration config;
 
@@ -457,4 +459,9 @@ public class QueryServicesOptions {
         return this;
     }
     
+    public QueryServicesOptions setExplainChunkCount(boolean showChunkCount) {
+        config.setBoolean(EXPLAIN_CHUNK_COUNT_ATTRIB, showChunkCount);
+        return this;
+    }
+    
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4ed86c0c/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 2af2666..c1b7f99 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
@@ -52,6 +52,7 @@ public final class QueryServicesTestImpl extends BaseQueryServicesImpl {
     public static final long DEFAULT_MAX_SERVER_METADATA_CACHE_SIZE =  1024L*1024L*4L; // 4 Mb
     public static final long DEFAULT_MAX_CLIENT_METADATA_CACHE_SIZE =  1024L*1024L*2L; // 2 Mb
     public static final int DEFAULT_MIN_STATS_UPDATE_FREQ_MS = 0;
+    public static final boolean DEFAULT_EXPLAIN_CHUNK_COUNT = false; // TODO: update explain plans in test and set to true
     
     /**
      * Set number of salt buckets lower for sequence table during testing, as a high
@@ -66,6 +67,7 @@ public final class QueryServicesTestImpl extends BaseQueryServicesImpl {
     
     private static QueryServicesOptions getDefaultServicesOptions() {
     	return withDefaults()
+    	        .setExplainChunkCount(DEFAULT_EXPLAIN_CHUNK_COUNT)
     	        .setSequenceSaltBuckets(DEFAULT_SEQUENCE_TABLE_SALT_BUCKETS)
                 .setMinStatsUpdateFrequencyMs(DEFAULT_MIN_STATS_UPDATE_FREQ_MS)
                 .setThreadPoolSize(DEFAULT_THREAD_POOL_SIZE)