From commits-return-19242-archive-asf-public=cust-asf.ponee.io@phoenix.apache.org Wed Jan 24 22:13:53 2018 Return-Path: X-Original-To: archive-asf-public@eu.ponee.io Delivered-To: archive-asf-public@eu.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by mx-eu-01.ponee.io (Postfix) with ESMTP id 73A6D180630 for ; Wed, 24 Jan 2018 22:13:53 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id 64034160C3C; Wed, 24 Jan 2018 21:13:53 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 5BDEF160C39 for ; Wed, 24 Jan 2018 22:13:52 +0100 (CET) Received: (qmail 61667 invoked by uid 500); 24 Jan 2018 21:13:51 -0000 Mailing-List: contact commits-help@phoenix.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@phoenix.apache.org Delivered-To: mailing list commits@phoenix.apache.org Received: (qmail 61658 invoked by uid 99); 24 Jan 2018 21:13:51 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 24 Jan 2018 21:13:51 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 66953DFC32; Wed, 24 Jan 2018 21:13:51 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: tdsilva@apache.org To: commits@phoenix.apache.org Message-Id: <9dd4d78508e1445e8921e9842f9c74ff@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: phoenix git commit: PHOENIX-4548 UpgradeUtil.mapChildViewsToNamespace does not handle multi-tenant views that have the same name Date: Wed, 24 Jan 2018 21:13:51 +0000 (UTC) Repository: phoenix Updated Branches: refs/heads/master 65f91a11d -> 3a6c76f12 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/3a6c76f1 Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/3a6c76f1 Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/3a6c76f1 Branch: refs/heads/master Commit: 3a6c76f122d7df1aa6fe9eb76f100ea23d298a03 Parents: 65f91a1 Author: Thomas D'Silva Authored: Mon Jan 22 11:08:03 2018 -0800 Committer: Thomas D'Silva Committed: Wed Jan 24 09:56:03 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/3a6c76f1/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/3a6c76f1/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 c4c2834..f09590d 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 @@ -1828,6 +1828,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()); @@ -1900,6 +1902,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(); @@ -1932,22 +1935,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)