You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by sa...@apache.org on 2016/11/04 22:13:41 UTC

[14/50] [abbrv] phoenix git commit: PHOENIX-3421 Column name lookups fail when on an indexed table

PHOENIX-3421 Column name lookups fail when on an indexed table


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

Branch: refs/heads/encodecolumns2
Commit: fc3af300faa4a8af4025aff5696b8b28d3193ec9
Parents: bb88e9f
Author: James Taylor <ja...@apache.org>
Authored: Thu Oct 27 23:09:09 2016 -0700
Committer: James Taylor <ja...@apache.org>
Committed: Thu Oct 27 23:13:07 2016 -0700

----------------------------------------------------------------------
 .../org/apache/phoenix/util/PhoenixRuntime.java | 34 +++++++++++++++
 .../phoenix/compile/QueryOptimizerTest.java     |  3 +-
 .../apache/phoenix/util/PhoenixRuntimeTest.java | 44 ++++++++++++++++++++
 3 files changed, 79 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/fc3af300/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java b/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
index 6027b95..b2f9ffc 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
@@ -804,6 +804,33 @@ public class PhoenixRuntime {
     private static String addQuotes(String str) {
         return str == null ? str : "\"" + str + "\"";
     }
+    
+    /**
+    * Get the column family, column name pairs that make up the row key of the table that will be queried.
+    * @param conn - connection used to generate the query plan. Caller should take care of closing the connection appropriately.
+    * @param plan - query plan to get info for.
+    * @return the pairs of column family name and column name columns in the data table that make up the row key for
+    * the table used in the query plan. Column family names are optional and hence the first part of the pair is nullable.
+    * Column names and family names are enclosed in double quotes to allow for case sensitivity and for presence of 
+    * special characters. Salting column and view index id column are not included. If the connection is tenant specific 
+    * and the table used by the query plan is multi-tenant, then the tenant id column is not included as well.
+    * @throws SQLException
+    */
+    public static List<Pair<String, String>> getPkColsForSql(Connection conn, QueryPlan plan) throws SQLException {
+        checkNotNull(plan);
+        checkNotNull(conn);
+        List<PColumn> pkColumns = getPkColumns(plan.getTableRef().getTable(), conn, true);
+        List<Pair<String, String>> columns = Lists.newArrayListWithExpectedSize(pkColumns.size());
+        String columnName;
+        String familyName;
+        for (PColumn pCol : pkColumns ) {
+            columnName = addQuotes(pCol.getName().getString());
+            familyName = pCol.getFamilyName() != null ? addQuotes(pCol.getFamilyName().getString()) : null;
+            columns.add(new Pair<String, String>(familyName, columnName));
+        }
+        return columns;
+    }
+
     /**
      *
      * @param columns - Initialized empty list to be filled with the pairs of column family name and column name for columns that are used 
@@ -818,6 +845,7 @@ public class PhoenixRuntime {
      * names correspond to the index table.
      * @throws SQLException
      */
+    @Deprecated
     public static void getPkColsForSql(List<Pair<String, String>> columns, QueryPlan plan, Connection conn, boolean forDataTable) throws SQLException {
         checkNotNull(columns);
         checkNotNull(plan);
@@ -846,6 +874,7 @@ public class PhoenixRuntime {
      * types correspond to the index table.
      * @throws SQLException
      */
+    @Deprecated
     public static void getPkColsDataTypesForSql(List<Pair<String, String>> columns, List<String> dataTypes, QueryPlan plan, Connection conn, boolean forDataTable) throws SQLException {
         checkNotNull(columns);
         checkNotNull(dataTypes);
@@ -1025,6 +1054,11 @@ public class PhoenixRuntime {
         // normalize and remove quotes from family and column names before looking up.
         familyName = SchemaUtil.normalizeIdentifier(familyName);
         columnName = SchemaUtil.normalizeIdentifier(columnName);
+        // Column names are always for the data table, so we must translate them if
+        // we're dealing with an index table.
+        if (table.getType() == PTableType.INDEX) {
+            columnName = IndexUtil.getIndexColumnName(familyName, columnName);
+        }
         PColumn pColumn = null;
         if (familyName != null) {
             PColumnFamily family = table.getColumnFamily(familyName);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fc3af300/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryOptimizerTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryOptimizerTest.java b/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryOptimizerTest.java
index 47dfcd3..b3a845c 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryOptimizerTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryOptimizerTest.java
@@ -641,8 +641,7 @@ public class QueryOptimizerTest extends BaseConnectionlessQueryTest {
         Connection conn = stmt.getConnection();
         QueryPlan plan = PhoenixRuntime.getOptimizedQueryPlan(stmt);
         
-        List<Pair<String, String>> columns = new ArrayList<Pair<String, String>>();
-        PhoenixRuntime.getPkColsForSql(columns, plan, conn, true);
+        List<Pair<String, String>> columns = PhoenixRuntime.getPkColsForSql(conn, plan);
         assertEquals(expectedPkCols, Joiner.on(",").join(getColumnNames(columns)));
         List<String> dataTypes = new ArrayList<String>();
         columns = new ArrayList<Pair<String,String>>();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fc3af300/phoenix-core/src/test/java/org/apache/phoenix/util/PhoenixRuntimeTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/util/PhoenixRuntimeTest.java b/phoenix-core/src/test/java/org/apache/phoenix/util/PhoenixRuntimeTest.java
index fe81120..783ab17 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/util/PhoenixRuntimeTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/util/PhoenixRuntimeTest.java
@@ -18,6 +18,7 @@
 
 package org.apache.phoenix.util;
 
+import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
@@ -26,6 +27,7 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.sql.Connection;
+import java.sql.Date;
 import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.SQLException;
@@ -39,6 +41,7 @@ import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.compile.QueryPlan;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.query.BaseConnectionlessQueryTest;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.PTable;
@@ -102,6 +105,47 @@ public class PhoenixRuntimeTest extends BaseConnectionlessQueryTest {
     }
     
     @Test
+    public void testGetPkColsEncodeDecode() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        String ddl = "CREATE TABLE t (\n" + 
+                "TENANT_ID VARCHAR NOT NULL,\n" +
+                "PARENT_ID CHAR(15) NOT NULL,\n" + 
+                "CREATED_DATE DATE NOT NULL,\n" + 
+                "ENTITY_HISTORY_ID CHAR(15) NOT NULL,\n" + 
+                "DATA_TYPE VARCHAR,\n" + 
+                "OLDVAL_STRING VARCHAR,\n" + 
+                "NEWVAL_STRING VARCHAR\n" + 
+                "CONSTRAINT PK PRIMARY KEY(TENANT_ID, PARENT_ID, CREATED_DATE DESC, ENTITY_HISTORY_ID))"
+                + " MULTI_TENANT = true, IMMUTABLE_ROWS = true";
+        conn.createStatement().execute(ddl);
+        String indexDDL = "CREATE INDEX i ON t (CREATED_DATE, PARENT_ID) INCLUDE (DATA_TYPE, OLDVAL_STRING, NEWVAL_STRING)";
+        conn.createStatement().execute(indexDDL);
+        
+        String tenantId = "111111111111111";
+        String parentId = "222222222222222";
+        Date createdDate = new Date(System.currentTimeMillis());
+        String ehId = "333333333333333";
+        
+        Object[] values = new Object[] {tenantId, createdDate, parentId, ehId};
+        QueryPlan plan = conn.createStatement().unwrap(PhoenixStatement.class).optimizeQuery("SELECT PARENT_ID FROM T WHERE CREATED_DATE > CURRENT_DATE()-1 AND TENANT_ID = '111111111111111'");
+        List<Pair<String,String>> pkColumns = PhoenixRuntime.getPkColsForSql(conn, plan);
+        String fullTableName = plan.getTableRef().getTable().getName().getString();
+        assertEquals("I", fullTableName);
+        byte[] encodedValues = PhoenixRuntime.encodeValues(conn, fullTableName, values, pkColumns);
+        Object[] decodedValues = PhoenixRuntime.decodeValues(conn, fullTableName, encodedValues, pkColumns);
+        assertArrayEquals(values, decodedValues);
+        
+        plan = conn.createStatement().unwrap(PhoenixStatement.class).optimizeQuery("SELECT /*+ NO_INDEX */ ENTITY_HISTORY_ID FROM T");
+        pkColumns = PhoenixRuntime.getPkColsForSql(conn, plan);
+        values = new Object[] {tenantId, parentId, createdDate, ehId};
+        fullTableName = plan.getTableRef().getTable().getName().getString();
+        assertEquals("T", fullTableName);
+        encodedValues = PhoenixRuntime.encodeValues(conn, fullTableName, values, pkColumns);
+        decodedValues = PhoenixRuntime.decodeValues(conn, fullTableName, encodedValues, pkColumns);
+        assertArrayEquals(values, decodedValues);
+    }
+    
+    @Test
     public void testGetPkColsDataTypes() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl(), new Properties());
         int i = 0;