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:42 UTC

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

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)