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 2016/02/12 05:14:45 UTC

[1/2] phoenix git commit: PHOENIX-2681 Avoid usage of HashSet in guideposts selection

Repository: phoenix
Updated Branches:
  refs/heads/master decbfe306 -> 0c21539cc


PHOENIX-2681 Avoid usage of HashSet<byte[]> in guideposts selection


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

Branch: refs/heads/master
Commit: 18f7a69452eec7fd5fde38953510600c4a060151
Parents: decbfe3
Author: James Taylor <jt...@salesforce.com>
Authored: Thu Feb 11 20:09:16 2016 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Thu Feb 11 20:14:31 2016 -0800

----------------------------------------------------------------------
 .../phoenix/end2end/MultiCfQueryExecIT.java     | 51 ++++++++++++++++++++
 .../phoenix/iterate/BaseResultIterators.java    | 35 ++++++--------
 2 files changed, 66 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/18f7a694/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 f5566ce..2b14fe9 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
@@ -52,6 +52,7 @@ public class MultiCfQueryExecIT extends BaseOwnClusterClientManagedTimeIT {
         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.QUEUE_SIZE_ATTRIB, Long.toString(200));
         setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
     }
     
@@ -184,6 +185,56 @@ public class MultiCfQueryExecIT extends BaseOwnClusterClientManagedTimeIT {
     }
 
     @Test
+    public void testGuidePostsForMultiCFsOverUnevenDistrib() throws Exception {
+        long ts = nextTimestamp();
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 10));
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        
+        conn.createStatement().execute("CREATE TABLE T_6CF (K1 CHAR(1) NOT NULL, "
+                + "K2 VARCHAR NOT NULL, "
+                + "CF1.A INTEGER, "
+                + "CF2.B INTEGER, "
+                + "CF3.C INTEGER, "
+                + "CF4.D INTEGER, "
+                + "CF5.E INTEGER, "
+                + "CF6.F INTEGER "
+                + "CONSTRAINT PK PRIMARY KEY (K1,K2)) SPLIT ON ('B','C','D')");
+
+        conn.close();
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 20));
+        conn = DriverManager.getConnection(getUrl(), props);
+        for (int i = 0; i < 100; i++) {
+            String upsert = "UPSERT INTO T_6CF(K1,K2,A) VALUES('" + Character.toString((char)('A'+i%10)) + "','" + (i*10) + "'," + i + ")";
+            conn.createStatement().execute(upsert);
+            if (i % 10 == 0) {
+                conn.createStatement().execute("UPSERT INTO T_6CF(K1,K2,F) VALUES('" + Character.toString((char)('A'+i%10)) + "','" + (i*10) + "'," + (i * 10) + ")");
+            }
+        }
+        conn.commit();
+        conn.close();
+        
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 40));
+        conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            analyzeTable(getUrl(), ts + 30, "T_6CF");
+            PreparedStatement statement = conn.prepareStatement("select count(*) from T_6CF where f < 400");
+            ResultSet rs = statement.executeQuery();
+            assertTrue(rs.next());
+            assertEquals(4, rs.getLong(1));
+            assertFalse(rs.next());
+            List<KeyRange> splits = getAllSplits(conn, "T_6CF", "f < 400", "COUNT(*)");
+            // Uses less populated column f
+            assertEquals(14, splits.size());
+            // Uses more populated column a
+            splits = getAllSplits(conn, "T_6CF", "a < 80", "COUNT(*)");
+            assertEquals(104, splits.size());
+        } finally {
+            conn.close();
+        }
+    }
+
+    @Test
     public void testGuidePostsRetrievedForMultiCF() throws Exception {
       Connection conn;
       PreparedStatement stmt;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/18f7a694/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
index 3a3d1f2..2352e94 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
@@ -30,7 +30,6 @@ import java.io.EOFException;
 import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Collections;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -82,8 +81,8 @@ import org.apache.phoenix.schema.TableRef;
 import org.apache.phoenix.schema.stats.GuidePostsInfo;
 import org.apache.phoenix.schema.stats.PTableStats;
 import org.apache.phoenix.util.ByteUtil;
-import org.apache.phoenix.util.PrefixByteCodec;
 import org.apache.phoenix.util.LogUtil;
+import org.apache.phoenix.util.PrefixByteCodec;
 import org.apache.phoenix.util.PrefixByteDecoder;
 import org.apache.phoenix.util.SQLCloseables;
 import org.apache.phoenix.util.ScanUtil;
@@ -382,24 +381,17 @@ public abstract class BaseResultIterators extends ExplainTable implements Result
             // For sure we can get the defaultCF from the table
             gps = getDefaultFamilyGuidePosts(guidePostMap, defaultCF);
         } else {
-            byte[] familyInWhere = null;
-            if (!whereConditions.isEmpty()) {
-                if (whereConditions.contains(defaultCF)) {
-                    gps = getDefaultFamilyGuidePosts(guidePostMap, defaultCF);
+            if (whereConditions.isEmpty() || whereConditions.contains(defaultCF)) {
+                gps = getDefaultFamilyGuidePosts(guidePostMap, defaultCF);
+            } else {
+                byte[] familyInWhere = whereConditions.iterator().next();
+                GuidePostsInfo guidePostsInfo = guidePostMap.get(familyInWhere);
+                if (guidePostsInfo != null) {
+                    gps = guidePostsInfo;
                 } else {
-                    familyInWhere = whereConditions.iterator().next();
-                    if (familyInWhere != null) {
-                        GuidePostsInfo guidePostsInfo = guidePostMap.get(familyInWhere);
-                        if (guidePostsInfo != null) {
-                            gps = guidePostsInfo;
-                        } else {
-                            // As there are no guideposts collected for the where family we go with the default CF
-                            gps = getDefaultFamilyGuidePosts(guidePostMap, defaultCF);
-                        }
-                    }
+                    // As there are no guideposts collected for the where family we go with the default CF
+                    gps = getDefaultFamilyGuidePosts(guidePostMap, defaultCF);
                 }
-            } else {
-                gps = getDefaultFamilyGuidePosts(guidePostMap, defaultCF);
             }
         }
         if (gps == null) { return GuidePostsInfo.EMPTY_GUIDEPOST; }
@@ -460,9 +452,12 @@ public abstract class BaseResultIterators extends ExplainTable implements Result
         PTable table = getTable();
         boolean isSalted = table.getBucketNum() != null;
         boolean isLocalIndex = table.getIndexType() == IndexType.LOCAL;
-        HashSet<byte[]> whereConditions = new HashSet<byte[]>(context.getWhereConditionColumns().size());
+        TreeSet<byte[]> whereConditions = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
         for(Pair<byte[], byte[]> where : context.getWhereConditionColumns()) {
-          whereConditions.add(where.getFirst());
+            byte[] cf = where.getFirst();
+            if (cf != null) {
+                whereConditions.add(cf);
+            }
         }
         GuidePostsInfo gps = getGuidePosts(whereConditions);
         boolean traverseAllRegions = isSalted || isLocalIndex;


[2/2] phoenix git commit: PHOENIX-2658 When using QueryRunner API UNION ALL queries fail with NPE (Alicia Ying Shu)

Posted by ja...@apache.org.
PHOENIX-2658 When using QueryRunner API UNION ALL queries fail with NPE (Alicia Ying Shu)


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

Branch: refs/heads/master
Commit: 0c21539cc331b8d6ca144604cf899068ad74fb25
Parents: 18f7a69
Author: James Taylor <jt...@salesforce.com>
Authored: Thu Feb 11 20:10:23 2016 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Thu Feb 11 20:14:32 2016 -0800

----------------------------------------------------------------------
 .../org/apache/phoenix/end2end/UnionAllIT.java  | 49 +++++++++++++++++++-
 .../apache/phoenix/compile/QueryCompiler.java   |  3 +-
 2 files changed, 50 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/0c21539c/phoenix-core/src/it/java/org/apache/phoenix/end2end/UnionAllIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UnionAllIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UnionAllIT.java
index 6531129..b391dcc 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UnionAllIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UnionAllIT.java
@@ -40,7 +40,6 @@ import org.apache.phoenix.util.ReadOnlyProps;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-
 public class UnionAllIT extends BaseOwnClusterHBaseManagedTimeIT {
 
     @BeforeClass
@@ -679,4 +678,52 @@ public class UnionAllIT extends BaseOwnClusterHBaseManagedTimeIT {
             conn.close();
         }
     }
+
+    @Test
+    public void testParameterMetaDataNotNull() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+    
+        String ddl = "CREATE TABLE test_table " +
+                "  (a_string varchar not null, col1 integer" +
+                "  CONSTRAINT pk PRIMARY KEY (a_string))\n";
+        createTestTable(getUrl(), ddl);
+        String dml = "UPSERT INTO test_table VALUES(?, ?)";
+        PreparedStatement stmt = conn.prepareStatement(dml);
+        stmt.setString(1, "a");
+        stmt.setInt(2, 10);
+        stmt.execute();
+        conn.commit();
+
+        ddl = "CREATE TABLE b_table " +
+                "  (a_string varchar not null, col1 integer" +
+                "  CONSTRAINT pk PRIMARY KEY (a_string))\n";
+        createTestTable(getUrl(), ddl);
+        dml = "UPSERT INTO b_table VALUES(?, ?)";
+        stmt = conn.prepareStatement(dml);
+        stmt.setString(1, "b");
+        stmt.setInt(2, 20);
+        stmt.execute();
+        conn.commit();
+
+        String query = "select * from test_table union all select * from b_table";
+
+        try{
+            PreparedStatement pstmt = conn.prepareStatement(query);
+            assertTrue(pstmt.getParameterMetaData() != null);
+            ResultSet rs = pstmt.executeQuery();
+            assertTrue(rs.next());
+            assertEquals("a",rs.getString(1));
+            assertEquals(10,rs.getInt(2));
+            assertTrue(rs.next());
+            assertEquals("b",rs.getString(1));
+            assertEquals(20,rs.getInt(2));
+            assertFalse(rs.next()); 
+        } catch (Exception ex) {
+            ex.printStackTrace();
+        } finally {
+            conn.close();
+        }
+    } 
+
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/0c21539c/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java
index 70bb815..9e756c8 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java
@@ -182,7 +182,8 @@ public class QueryCompiler {
         StatementContext context = new StatementContext(statement, resolver, scan, sequenceManager);
 
         QueryPlan plan = compileSingleFlatQuery(context, select, statement.getParameters(), false, false, null, null, false);
-        plan =  new UnionPlan(context, select, tableRef, plan.getProjector(), plan.getLimit(), plan.getOrderBy(), GroupBy.EMPTY_GROUP_BY, plans, null); 
+        plan =  new UnionPlan(context, select, tableRef, plan.getProjector(), plan.getLimit(), plan.getOrderBy(), GroupBy.EMPTY_GROUP_BY, 
+                plans, context.getBindManager().getParameterMetaData()); 
         return plan;
     }