You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by td...@apache.org on 2018/01/24 21:13:40 UTC

[1/3] phoenix git commit: PHOENIX-4468 Looking up a parent index table of a child view from a different client fails

Repository: phoenix
Updated Branches:
  refs/heads/5.x-HBase-2.0 d1b5fd528 -> ab796a8f5


PHOENIX-4468 Looking up a parent index table of a child view from a different client fails


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

Branch: refs/heads/5.x-HBase-2.0
Commit: 6322b7a01f7b1cab66a4e1fdd3c3b1c4ed58e14f
Parents: d1b5fd5
Author: Thomas D'Silva <td...@apache.org>
Authored: Mon Dec 18 11:48:39 2017 -0800
Committer: Thomas D'Silva <td...@apache.org>
Committed: Wed Jan 24 10:04:15 2018 -0800

----------------------------------------------------------------------
 .../apache/phoenix/end2end/PhoenixDriverIT.java | 160 +++++++++++++++++++
 .../org/apache/phoenix/end2end/SequenceIT.java  |  29 ----
 phoenix-core/src/main/antlr3/PhoenixSQL.g       |   1 +
 .../apache/phoenix/schema/MetaDataClient.java   |   4 +-
 .../org/apache/phoenix/schema/PTableImpl.java   |   2 +-
 .../org/apache/phoenix/util/PhoenixRuntime.java |  32 +++-
 6 files changed, 189 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/6322b7a0/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixDriverIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixDriverIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixDriverIT.java
new file mode 100644
index 0000000..216653c
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixDriverIT.java
@@ -0,0 +1,160 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.end2end;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Properties;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.phoenix.jdbc.PhoenixDriver;
+import org.apache.phoenix.jdbc.PhoenixStatement;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.SchemaUtil;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class PhoenixDriverIT extends BaseUniqueNamesOwnClusterIT {
+    
+    private static HBaseTestingUtility hbaseTestUtil;
+    private static String zkQuorum;
+    
+    @BeforeClass
+    public static void setUp() throws Exception {
+        Configuration conf = HBaseConfiguration.create();
+        hbaseTestUtil = new HBaseTestingUtility(conf);
+        setUpConfigForMiniCluster(conf);
+        conf.set(QueryServices.EXTRA_JDBC_ARGUMENTS_ATTRIB, QueryServicesOptions.DEFAULT_EXTRA_JDBC_ARGUMENTS);
+        hbaseTestUtil.startMiniCluster();
+        // establish url and quorum. Need to use PhoenixDriver and not PhoenixTestDriver
+        zkQuorum = "localhost:" + hbaseTestUtil.getZkCluster().getClientPort();
+        url = PhoenixRuntime.JDBC_PROTOCOL + PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR + zkQuorum;
+        DriverManager.registerDriver(PhoenixDriver.INSTANCE);
+    }
+    
+    public Connection createConnection(boolean isMultiTenant, boolean isDifferentClient) throws SQLException {
+        Properties props = new Properties();
+        props.setProperty(QueryServices.RETURN_SEQUENCE_VALUES_ATTRIB, "false");
+        // force the use of ConnectionQueryServicesImpl instead of ConnectionQueryServicesTestImpl
+        props.put(QueryServices.EXTRA_JDBC_ARGUMENTS_ATTRIB,
+            QueryServicesOptions.DEFAULT_EXTRA_JDBC_ARGUMENTS);
+        if (isMultiTenant)
+            props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, "tenant1");
+        StringBuilder sb = new StringBuilder(url);
+        if (isMultiTenant)
+            sb.append(PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR + "Client2");
+        return DriverManager.getConnection(sb.toString(), props);
+    }
+    
+    @Test
+    public void testReturnAllSequencesNotCalledForNoOpenConnections() throws Exception {
+        String schemaName = "S";
+        String sequenceNameWithoutSchema = generateUniqueSequenceName();
+        String sequenceName = SchemaUtil.getTableName(schemaName, sequenceNameWithoutSchema);
+        
+        Connection conn = createConnection(false, false);
+        conn.createStatement().execute("CREATE SEQUENCE " + sequenceName + " START WITH 3 INCREMENT BY 2 CACHE 5");
+        
+        String query = "SELECT NEXT VALUE FOR " + sequenceName ;
+        ResultSet rs = conn.prepareStatement(query).executeQuery();
+        assertTrue(rs.next());
+        assertEquals(3, rs.getInt(1));
+        assertFalse(rs.next());
+        rs = conn.prepareStatement(query).executeQuery();
+        assertTrue(rs.next());
+        assertEquals(5, rs.getInt(1));
+        assertFalse(rs.next());
+        conn.close();
+        
+        conn = createConnection(false, false);
+        // verify that calling close() does not return sequence values back to the server
+        query = "SELECT CURRENT_VALUE FROM \"SYSTEM\".\"SEQUENCE\" WHERE SEQUENCE_SCHEMA=? AND SEQUENCE_NAME=?";
+        PreparedStatement preparedStatement = conn.prepareStatement(query);
+        preparedStatement.setString(1, schemaName);
+        preparedStatement.setString(2, sequenceNameWithoutSchema);
+        rs = preparedStatement.executeQuery();
+        assertTrue(rs.next());
+        assertEquals(13, rs.getInt(1));
+        assertFalse(rs.next());
+        conn.close();
+    }
+    
+    @Test
+    public void testViewParentIndexLookupMutipleClients() throws Exception {
+        helpTestViewParentIndexLookupMutipleClients(false);
+    }
+    
+    @Test
+    public void testMulitTenantViewParentIndexLookupMutipleClients() throws Exception {
+        helpTestViewParentIndexLookupMutipleClients(true);
+    }
+    
+    public void helpTestViewParentIndexLookupMutipleClients(boolean isMultiTenant) throws Exception {
+        final String baseTableName = generateUniqueName();
+        final String baseTableIndexName = generateUniqueName();
+        final String viewName = generateUniqueName();
+        try (Connection globalConn = createConnection(false, false);
+                Connection conn1 = createConnection(isMultiTenant, false);
+                Connection conn2 = createConnection(isMultiTenant, false)) {
+            // create base table
+            String baseTableDdl = "CREATE TABLE " + baseTableName + " (" +
+                    ( isMultiTenant ? "TENANT_ID VARCHAR(1) NOT NULL," : "") +
+                    "PK CHAR(1) NOT NULL," +
+                    "V1 CHAR(1)," +
+                    "V2 CHAR(1)," +
+                    "V3 CHAR(1)" + 
+                    "CONSTRAINT pk PRIMARY KEY (" + (isMultiTenant ? "TENANT_ID," : "") + " pk))";
+            globalConn.createStatement().execute(baseTableDdl);
+            
+            // create index on parent view
+            globalConn.createStatement().execute("CREATE INDEX " + baseTableIndexName + " ON " + baseTableName + " (V2) INCLUDE (v1, V3)");
+            
+            // create a view on the base table
+            String viewDDL = "CREATE VIEW " + viewName + " AS SELECT * FROM " + baseTableName + " WHERE V1 = 'X'";
+            conn1.createStatement().execute(viewDDL);
+            conn1.commit();
+
+            // ensure we can use parent table index
+            String sql = "SELECT V3 FROM " + viewName +" WHERE V2 = '3'";
+            PhoenixStatement stmt = conn1.createStatement().unwrap(PhoenixStatement.class);
+            stmt.executeQuery(sql);
+            PTable indexTable = stmt.getQueryPlan().getTableRef().getTable();
+            String tableName = indexTable.getName().getString();
+            String expectedTableName = baseTableIndexName + QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR + viewName;
+            assertEquals("Parent Index table is not used ", expectedTableName, tableName);
+            
+            // verify that we can look up the index using PhoenixRuntime from a different client
+            PTable table = PhoenixRuntime.getTable(conn2, tableName);
+            assertEquals(indexTable, table);
+        }
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6322b7a0/phoenix-core/src/it/java/org/apache/phoenix/end2end/SequenceIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SequenceIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SequenceIT.java
index 3014e45..9b870e1 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SequenceIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SequenceIT.java
@@ -1358,35 +1358,6 @@ public class SequenceIT extends ParallelStatsDisabledIT {
         assertEquals(5, rs.getInt(2));        
     }
     
-    @Test
-    public void testReturnAllSequencesNotCalledForNoOpenConnections() throws Exception {
-        String sequenceName = generateSequenceNameWithSchema();
-        String sequenceNameWithoutSchema = getNameWithoutSchema(sequenceName);
-        String schemaName = getSchemaName(sequenceName);
-        
-        conn.createStatement().execute("CREATE SEQUENCE " + sequenceName + " START WITH 3 INCREMENT BY 2 CACHE 5");
-        
-        String query = "SELECT NEXT VALUE FOR " + sequenceName ;
-        ResultSet rs = conn.prepareStatement(query).executeQuery();
-        assertTrue(rs.next());
-        assertEquals(3, rs.getInt(1));
-        assertFalse(rs.next());
-        rs = conn.prepareStatement(query).executeQuery();
-        assertTrue(rs.next());
-        assertEquals(5, rs.getInt(1));
-        assertFalse(rs.next());
-        
-        // verify that calling close() does not return sequence values back to the server
-        query = "SELECT CURRENT_VALUE FROM \"SYSTEM\".\"SEQUENCE\" WHERE SEQUENCE_SCHEMA=? AND SEQUENCE_NAME=?";
-        PreparedStatement preparedStatement = conn.prepareStatement(query);
-        preparedStatement.setString(1, schemaName);
-        preparedStatement.setString(2, sequenceNameWithoutSchema);
-        rs = preparedStatement.executeQuery();
-        assertTrue(rs.next());
-        assertEquals(13, rs.getInt(1));
-        assertFalse(rs.next());
-    }
-    
     private static String getSchemaName(String tableName) {
     	return tableName.substring(0, tableName.indexOf("."));
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6322b7a0/phoenix-core/src/main/antlr3/PhoenixSQL.g
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/antlr3/PhoenixSQL.g b/phoenix-core/src/main/antlr3/PhoenixSQL.g
index 93e0ede..f5eeb33 100644
--- a/phoenix-core/src/main/antlr3/PhoenixSQL.g
+++ b/phoenix-core/src/main/antlr3/PhoenixSQL.g
@@ -1084,6 +1084,7 @@ from_table_name returns [TableName ret]
 table_identifier returns [String ret]
     :   c=identifier {
            if (c.contains(QueryConstants.NAMESPACE_SEPARATOR) ) { throw new RuntimeException("Table or schema name cannot contain colon"); }
+           if (c.contains(QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR) ) { throw new RuntimeException("Table or schema name cannot contain hash"); }
            $ret = c;
     }
     ;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6322b7a0/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 ee9ddc0..46998c4 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
@@ -846,8 +846,8 @@ public class MetaDataClient {
             if (containsAllReqdCols) {
                 // Tack on view statement to index to get proper filtering for view
                 String viewStatement = IndexUtil.rewriteViewStatement(connection, index, parentTable, view.getViewStatement());
-                PName modifiedIndexName = PNameFactory.newName(index.getSchemaName().getString() + QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR
-                        + index.getName().getString() + QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR + view.getName().getString());
+                PName modifiedIndexName = PNameFactory.newName(index.getName().getString() 
+                    + QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR + view.getName().getString());
                 // add the index table with a new name so that it does not conflict with the existing index table
                 // also set update cache frequency to never since the renamed index is not present on the server
                 indexesToAdd.add(PTableImpl.makePTable(index, modifiedIndexName, viewStatement, Long.MAX_VALUE, view.getTenantId()));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6322b7a0/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
index 0b48376..13d0da6 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
@@ -267,7 +267,7 @@ public class PTableImpl implements PTable {
     
     public static PTableImpl makePTable(PTable table, PName tableName, long timeStamp, List<PName> physicalNames, List<PTable> indexes, String viewStatement, long updateCacheFrequency, PName tenantId) throws SQLException {
             return new PTableImpl(
-                    tenantId, table.getSchemaName(), tableName, table.getType(), table.getIndexState(), timeStamp,
+                    tenantId, PNameImpl.EMPTY_NAME, tableName, table.getType(), table.getIndexState(), timeStamp,
                     table.getSequenceNumber(), table.getPKName(), table.getBucketNum(), getColumnsToClone(table), table.getParentSchemaName(), table.getParentTableName(),
                     indexes, table.isImmutableRows(), physicalNames, table.getDefaultFamilyName(), viewStatement,
                     table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), table.getIndexType(),

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6322b7a0/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 6b5a73a..26a1dcf 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
@@ -429,8 +429,15 @@ public class PhoenixRuntime {
         return result.getTable();
 
     }
+    
     /**
-     * 
+     * Returns the table if it is found in the connection metadata cache. If the metadata of this
+     * table has changed since it was put in the cache these changes will not necessarily be
+     * reflected in the returned table. If the table is not found, makes a call to the server to
+     * fetch the latest metadata of the table. This is different than how a table is resolved when
+     * it is referenced from a query (a call is made to the server to fetch the latest metadata of the table
+     * depending on the UPDATE_CACHE_FREQUENCY property)
+     * See https://issues.apache.org/jira/browse/PHOENIX-4475
      * @param conn
      * @param name requires a pre-normalized table name or a pre-normalized schema and table name
      * @return
@@ -442,13 +449,24 @@ public class PhoenixRuntime {
         try {
             table = pconn.getTable(new PTableKey(pconn.getTenantId(), name));
         } catch (TableNotFoundException e) {
-            String schemaName = SchemaUtil.getSchemaNameFromFullName(name);
-            String tableName = SchemaUtil.getTableNameFromFullName(name);
-            MetaDataMutationResult result = new MetaDataClient(pconn).updateCache(schemaName, tableName);
-            if (result.getMutationCode() != MutationCode.TABLE_ALREADY_EXISTS) {
-                throw e;
+            // parent indexes on child view metadata rows are not present on the server
+            if (name.contains(QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR)) {
+                String viewName =
+                        SchemaUtil.getTableNameFromFullName(name,
+                            QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR);
+                // resolve the view which should also load any parent indexes
+                getTable(conn, viewName);
+                table = pconn.getTable(new PTableKey(pconn.getTenantId(), name));
+            } else {
+                String schemaName = SchemaUtil.getSchemaNameFromFullName(name);
+                String tableName = SchemaUtil.getTableNameFromFullName(name);
+                MetaDataMutationResult result =
+                        new MetaDataClient(pconn).updateCache(schemaName, tableName);
+                if (result.getMutationCode() != MutationCode.TABLE_ALREADY_EXISTS) {
+                    throw e;
+                }
+                table = result.getTable();
             }
-            table = result.getTable();
         }
         return table;
     }


[2/3] phoenix git commit: PHOENIX-4468 Looking up a parent index table of a child view from a different client fails (addendum)

Posted by td...@apache.org.
PHOENIX-4468 Looking up a parent index table of a child view from a different client fails (addendum)


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

Branch: refs/heads/5.x-HBase-2.0
Commit: 4031a9c076664c83b6f8a87ca3a149326ba1facd
Parents: 6322b7a
Author: Thomas D'Silva <td...@apache.org>
Authored: Wed Dec 20 15:47:19 2017 -0800
Committer: Thomas D'Silva <td...@apache.org>
Committed: Wed Jan 24 10:04:29 2018 -0800

----------------------------------------------------------------------
 phoenix-core/src/main/antlr3/PhoenixSQL.g                          | 1 -
 .../src/main/java/org/apache/phoenix/schema/PTableImpl.java        | 2 +-
 2 files changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/4031a9c0/phoenix-core/src/main/antlr3/PhoenixSQL.g
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/antlr3/PhoenixSQL.g b/phoenix-core/src/main/antlr3/PhoenixSQL.g
index f5eeb33..93e0ede 100644
--- a/phoenix-core/src/main/antlr3/PhoenixSQL.g
+++ b/phoenix-core/src/main/antlr3/PhoenixSQL.g
@@ -1084,7 +1084,6 @@ from_table_name returns [TableName ret]
 table_identifier returns [String ret]
     :   c=identifier {
            if (c.contains(QueryConstants.NAMESPACE_SEPARATOR) ) { throw new RuntimeException("Table or schema name cannot contain colon"); }
-           if (c.contains(QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR) ) { throw new RuntimeException("Table or schema name cannot contain hash"); }
            $ret = c;
     }
     ;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4031a9c0/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
index 13d0da6..0b48376 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
@@ -267,7 +267,7 @@ public class PTableImpl implements PTable {
     
     public static PTableImpl makePTable(PTable table, PName tableName, long timeStamp, List<PName> physicalNames, List<PTable> indexes, String viewStatement, long updateCacheFrequency, PName tenantId) throws SQLException {
             return new PTableImpl(
-                    tenantId, PNameImpl.EMPTY_NAME, tableName, table.getType(), table.getIndexState(), timeStamp,
+                    tenantId, table.getSchemaName(), tableName, table.getType(), table.getIndexState(), timeStamp,
                     table.getSequenceNumber(), table.getPKName(), table.getBucketNum(), getColumnsToClone(table), table.getParentSchemaName(), table.getParentTableName(),
                     indexes, table.isImmutableRows(), physicalNames, table.getDefaultFamilyName(), viewStatement,
                     table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), table.getIndexType(),


[3/3] phoenix git commit: PHOENIX-4548 UpgradeUtil.mapChildViewsToNamespace does not handle multi-tenant views that have the same name

Posted by td...@apache.org.
PHOENIX-4548 UpgradeUtil.mapChildViewsToNamespace does not handle multi-tenant views that have the same name


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

Branch: refs/heads/5.x-HBase-2.0
Commit: ab796a8f525c54d78b0c0414640544875feb2b62
Parents: 4031a9c
Author: Thomas D'Silva <td...@apache.org>
Authored: Mon Jan 22 11:08:03 2018 -0800
Committer: Thomas D'Silva <td...@apache.org>
Committed: Wed Jan 24 10:04:38 2018 -0800

----------------------------------------------------------------------
 .../apache/phoenix/end2end/PhoenixDriverIT.java | 97 ++++++++++++++++++--
 .../org/apache/phoenix/util/UpgradeUtil.java    | 28 ++++--
 2 files changed, 110 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/ab796a8f/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixDriverIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixDriverIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixDriverIT.java
index 216653c..407e9cf 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixDriverIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixDriverIT.java
@@ -17,10 +17,17 @@
  */
 package org.apache.phoenix.end2end;
 
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.LINK_TYPE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_SCHEM;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TENANT_ID;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
+import java.io.IOException;
 import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.PreparedStatement;
@@ -31,14 +38,18 @@ import java.util.Properties;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
+import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDriver;
 import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTable.LinkType;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.UpgradeUtil;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -60,16 +71,16 @@ public class PhoenixDriverIT extends BaseUniqueNamesOwnClusterIT {
         DriverManager.registerDriver(PhoenixDriver.INSTANCE);
     }
     
-    public Connection createConnection(boolean isMultiTenant, boolean isDifferentClient) throws SQLException {
+    public Connection createConnection(String tenantId, boolean isDifferentClient) throws SQLException {
         Properties props = new Properties();
         props.setProperty(QueryServices.RETURN_SEQUENCE_VALUES_ATTRIB, "false");
         // force the use of ConnectionQueryServicesImpl instead of ConnectionQueryServicesTestImpl
         props.put(QueryServices.EXTRA_JDBC_ARGUMENTS_ATTRIB,
             QueryServicesOptions.DEFAULT_EXTRA_JDBC_ARGUMENTS);
-        if (isMultiTenant)
-            props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, "tenant1");
+        if (tenantId!=null)
+            props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId);
         StringBuilder sb = new StringBuilder(url);
-        if (isMultiTenant)
+        if (isDifferentClient)
             sb.append(PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR + "Client2");
         return DriverManager.getConnection(sb.toString(), props);
     }
@@ -80,7 +91,7 @@ public class PhoenixDriverIT extends BaseUniqueNamesOwnClusterIT {
         String sequenceNameWithoutSchema = generateUniqueSequenceName();
         String sequenceName = SchemaUtil.getTableName(schemaName, sequenceNameWithoutSchema);
         
-        Connection conn = createConnection(false, false);
+        Connection conn = createConnection(null, false);
         conn.createStatement().execute("CREATE SEQUENCE " + sequenceName + " START WITH 3 INCREMENT BY 2 CACHE 5");
         
         String query = "SELECT NEXT VALUE FOR " + sequenceName ;
@@ -94,7 +105,7 @@ public class PhoenixDriverIT extends BaseUniqueNamesOwnClusterIT {
         assertFalse(rs.next());
         conn.close();
         
-        conn = createConnection(false, false);
+        conn = createConnection(null, false);
         // verify that calling close() does not return sequence values back to the server
         query = "SELECT CURRENT_VALUE FROM \"SYSTEM\".\"SEQUENCE\" WHERE SEQUENCE_SCHEMA=? AND SEQUENCE_NAME=?";
         PreparedStatement preparedStatement = conn.prepareStatement(query);
@@ -121,9 +132,9 @@ public class PhoenixDriverIT extends BaseUniqueNamesOwnClusterIT {
         final String baseTableName = generateUniqueName();
         final String baseTableIndexName = generateUniqueName();
         final String viewName = generateUniqueName();
-        try (Connection globalConn = createConnection(false, false);
-                Connection conn1 = createConnection(isMultiTenant, false);
-                Connection conn2 = createConnection(isMultiTenant, false)) {
+        try (Connection globalConn = createConnection(null, false);
+                Connection conn1 = createConnection("tenant1", false);
+                Connection conn2 = createConnection("tenant1", false)) {
             // create base table
             String baseTableDdl = "CREATE TABLE " + baseTableName + " (" +
                     ( isMultiTenant ? "TENANT_ID VARCHAR(1) NOT NULL," : "") +
@@ -157,4 +168,72 @@ public class PhoenixDriverIT extends BaseUniqueNamesOwnClusterIT {
         }
     }
     
+    @Test
+    public void testMapMultiTenantTableToNamespaceDuringUpgrade() throws SQLException,
+            SnapshotCreationException, IllegalArgumentException, IOException, InterruptedException {
+        String schemaName = "S_" + generateUniqueName();
+        String tableName = "T_" + generateUniqueName();
+        String phoenixFullTableName = SchemaUtil.getTableName(schemaName, tableName);
+        String viewName1 = "VC_" + generateUniqueName();
+        String viewName2 = "VB_" + generateUniqueName();
+
+        try (Connection conn = createConnection(null, false)) {
+            conn.createStatement().execute("CREATE TABLE " + phoenixFullTableName
+                    + "(k VARCHAR not null, v INTEGER not null, f INTEGER, g INTEGER NULL, h INTEGER NULL CONSTRAINT pk PRIMARY KEY(k,v)) MULTI_TENANT=true");
+        }
+        try (Connection conn = createConnection("tenant1", false)) {
+            // create view
+            conn.createStatement().execute("CREATE VIEW " + schemaName + "." + viewName1
+                    + " (col VARCHAR) AS SELECT * FROM " + phoenixFullTableName);
+            // create child view
+            conn.createStatement().execute("CREATE VIEW " + schemaName + "." + viewName2
+                    + " (col2 VARCHAR) AS SELECT * FROM " + schemaName + "." + viewName1);
+        }
+
+        String tenant2 = "tenant2";
+        try (Connection conn = createConnection(tenant2, false)) {
+            // creating another view in a second tenant but same view name
+            conn.createStatement().execute("CREATE VIEW " + schemaName + "." + viewName1
+                    + " (col VARCHAR) AS SELECT * FROM " + phoenixFullTableName);
+            // creating child view with a second tenant
+            conn.createStatement().execute("CREATE VIEW " + schemaName + "." + viewName2
+                    + " (col2 VARCHAR) AS SELECT * FROM " + schemaName + "." + viewName1);
+        }
+
+        try (Connection conn = createConnection(null, true)) {
+            String url = conn.unwrap(PhoenixConnection.class).getURL();
+            Properties props = new Properties();
+            props.setProperty(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.toString(true));
+            props.setProperty(QueryServices.IS_SYSTEM_TABLE_MAPPED_TO_NAMESPACE,
+                Boolean.toString(false));
+            try (PhoenixConnection phxConn =
+                    DriverManager.getConnection(url, props).unwrap(PhoenixConnection.class)) {
+                UpgradeUtil.upgradeTable(phxConn, phoenixFullTableName);
+                UpgradeUtil.mapChildViewsToNamespace(phxConn, phoenixFullTableName, props);
+            }
+
+            // verify physical table link
+            String physicalTableName =
+                    SchemaUtil.getPhysicalHBaseTableName(schemaName, tableName, true).getString();
+            assertEquals(physicalTableName, getPhysicalTable(conn, tenant2, schemaName, viewName1));
+            assertEquals(physicalTableName, getPhysicalTable(conn, tenant2, schemaName, viewName2));
+        }
+    }
+
+    private String getPhysicalTable(Connection conn, String tenantId, String schemeName, String tableName) throws SQLException {
+        String query =
+                "SELECT COLUMN_FAMILY FROM " + SYSTEM_CATALOG_SCHEMA + "." + SYSTEM_CATALOG_TABLE
+                        + " WHERE " + TENANT_ID + "=? AND " + TABLE_SCHEM + "=? AND " + TABLE_NAME
+                        + "=? AND " + LINK_TYPE + "="
+                        + LinkType.PHYSICAL_TABLE.getSerializedValue();
+        PreparedStatement stmt = conn.prepareStatement(query);
+        stmt.setString(1, tenantId);
+        stmt.setString(2, schemeName);
+        stmt.setString(3, tableName);
+        ResultSet rs = stmt.executeQuery();
+        assertTrue(rs.next());
+        String physicalTableName = rs.getString(1);
+        assertFalse(rs.next());
+        return physicalTableName;
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ab796a8f/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
index bb9ba3b..b34006e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
@@ -1831,6 +1831,8 @@ public class UpgradeUtil {
             String newPhysicalTablename = SchemaUtil.normalizeIdentifier(
                     SchemaUtil.getPhysicalTableName(oldPhysicalName, readOnlyProps).getNameAsString());
             logger.info(String.format("Upgrading %s %s..", table.getType(), tableName));
+            logger.info(String.format("oldPhysicalName %s newPhysicalTablename %s..", oldPhysicalName, newPhysicalTablename));
+            logger.info(String.format("teanantId %s..", conn.getTenantId()));
             // Upgrade the data or main table
             mapTableToNamespace(admin, metatable, tableName, newPhysicalTablename, readOnlyProps,
                     PhoenixRuntime.getCurrentScn(readOnlyProps), tableName, table.getType(),conn.getTenantId());
@@ -1903,6 +1905,7 @@ public class UpgradeUtil {
                 throw new RuntimeException("Error: problem occured during upgrade. Table is not upgraded successfully");
             }
             if (table.getType() == PTableType.VIEW) {
+                logger.info(String.format("Updating link information for view '%s' ..", table.getTableName()));
                 updateLink(conn, oldPhysicalName, newPhysicalTablename,table.getSchemaName(),table.getTableName());
                 conn.commit();
 
@@ -1935,22 +1938,35 @@ public class UpgradeUtil {
         MetaDataUtil.deleteViewIndexSequences(connection, oldPhysicalName, false);
     }
 
-    private static void updateLink(PhoenixConnection conn, String srcTableName, String destTableName, PName schemaName, PName tableName)
-            throws SQLException {
-        PreparedStatement updateLinkStatment = conn.prepareStatement(String.format(UPDATE_LINK,destTableName));
+    private static void updateLink(PhoenixConnection conn, String srcTableName,
+            String destTableName, PName schemaName, PName tableName) throws SQLException {
+        String updateLinkSql = String.format(UPDATE_LINK, destTableName);
+        boolean hasTenantId = conn.getTenantId() != null;
+        if (hasTenantId) {
+            updateLinkSql += " AND TENANT_ID  = ? ";
+        }
+        PreparedStatement updateLinkStatment = conn.prepareStatement(updateLinkSql);
         updateLinkStatment.setString(1, schemaName.getString());
         updateLinkStatment.setString(2, schemaName.getString());
         updateLinkStatment.setString(3, tableName.getString());
         updateLinkStatment.setString(4, srcTableName);
-        
+        if (hasTenantId) {
+            updateLinkStatment.setString(5, conn.getTenantId().getString());
+        }
         updateLinkStatment.execute();
-        PreparedStatement deleteLinkStatment = conn.prepareStatement(DELETE_LINK);
+        String deleteLinkSql = DELETE_LINK;
+        if (hasTenantId) {
+            deleteLinkSql += (" AND TENANT_ID  = ? ");
+        }
+        PreparedStatement deleteLinkStatment = conn.prepareStatement(deleteLinkSql);
         deleteLinkStatment.setString(1, schemaName.getString());
         deleteLinkStatment.setString(2, schemaName.getString());
         deleteLinkStatment.setString(3, tableName.getString());
         deleteLinkStatment.setString(4, srcTableName);
+        if (hasTenantId) {
+            deleteLinkStatment.setString(5, conn.getTenantId().getString());
+        }
         deleteLinkStatment.execute();
-        
     }
     
     public static void mapChildViewsToNamespace(PhoenixConnection conn, String table, Properties props)