This is an automated email from the ASF dual-hosted git repository. chinmayskulkarni pushed a commit to branch 4.x in repository https://gitbox.apache.org/repos/asf/phoenix.git
The following commit(s) were added to refs/heads/4.x by this push: new 11575c0 PHOENIX-6032: When phoenix.allow.system.catalog.rollback=true, a view still sees data from a column that was dropped 11575c0 is described below commit 11575c0ee931ae4e012ea2388068b317127f1882 Author: Chinmay Kulkarni <chinmayskulka...@gmail.com> AuthorDate: Thu Oct 29 17:21:06 2020 -0700 PHOENIX-6032: When phoenix.allow.system.catalog.rollback=true, a view still sees data from a column that was dropped --- .../AlterParentTableWithSysCatRollbackIT.java | 142 ---------- .../apache/phoenix/end2end/SystemCatalogIT.java | 140 ---------- .../end2end/SystemCatalogRollbackEnabledIT.java | 298 +++++++++++++++++++++ .../phoenix/coprocessor/MetaDataEndpointImpl.java | 4 +- .../java/org/apache/phoenix/util/ViewUtil.java | 20 +- 5 files changed, 316 insertions(+), 288 deletions(-) diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterParentTableWithSysCatRollbackIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterParentTableWithSysCatRollbackIT.java deleted file mode 100644 index 99a2124..0000000 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterParentTableWithSysCatRollbackIT.java +++ /dev/null @@ -1,142 +0,0 @@ -/* - * 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 com.google.common.collect.Maps; -import org.apache.phoenix.exception.SQLExceptionCode; -import org.apache.phoenix.query.BaseTest; -import org.apache.phoenix.query.QueryServices; -import org.apache.phoenix.util.ReadOnlyProps; -import org.apache.phoenix.util.SchemaUtil; -import org.junit.BeforeClass; -import org.junit.Test; -import org.junit.experimental.categories.Category; - -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.SQLException; -import java.util.Collections; -import java.util.Map; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; - -@Category(NeedsOwnMiniClusterTest.class) -public class AlterParentTableWithSysCatRollbackIT extends BaseTest { - - private String getJdbcUrl() { - return "jdbc:phoenix:localhost:" + getUtility().getZkCluster().getClientPort() - + ":/hbase"; - } - - @BeforeClass - public static synchronized void doSetup() throws Exception { - Map<String, String> serverProps = Maps.newHashMapWithExpectedSize(1); - serverProps.put(QueryServices.ALLOW_SPLITTABLE_SYSTEM_CATALOG_ROLLBACK, "true"); - setUpTestDriver(new ReadOnlyProps(serverProps.entrySet().iterator()), - new ReadOnlyProps(Collections.<String, String>emptyMap())); - } - - @Test - public void testAddColumnOnParentTableView() throws Exception { - try (Connection conn = DriverManager.getConnection(getJdbcUrl())) { - String parentTableName = SchemaUtil.getTableName(generateUniqueName(), - generateUniqueName()); - String parentViewName = SchemaUtil.getTableName(generateUniqueName(), - generateUniqueName()); - String childViewName = SchemaUtil.getTableName(generateUniqueName(), - generateUniqueName()); - // create parent table - String ddl = "CREATE TABLE " + parentTableName - + " (col1 INTEGER NOT NULL, col2 INTEGER " + "CONSTRAINT pk PRIMARY KEY (col1))"; - conn.createStatement().execute(ddl); - - // create view from table - ddl = "CREATE VIEW " + parentViewName + " AS SELECT * FROM " + parentTableName; - conn.createStatement().execute(ddl); - try { - ddl = "ALTER TABLE " + parentTableName + " ADD col4 INTEGER"; - conn.createStatement().execute(ddl); - fail("ALTER TABLE should not be allowed on parent table"); - } catch (SQLException e) { - assertEquals(SQLExceptionCode.CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode()); - } - - // create child view from above view - ddl = "CREATE VIEW " + childViewName + "(col3 INTEGER) AS SELECT * FROM " - + parentViewName; - conn.createStatement().execute(ddl); - try { - ddl = "ALTER VIEW " + parentViewName + " ADD col4 INTEGER"; - conn.createStatement().execute(ddl); - fail("ALTER VIEW should not be allowed on parent view"); - } catch (SQLException e) { - assertEquals(SQLExceptionCode.CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode()); - } - - // alter child view with add column should be allowed - ddl = "ALTER VIEW " + childViewName + " ADD col4 INTEGER"; - conn.createStatement().execute(ddl); - } - } - - @Test - public void testDropColumnOnParentTableView() throws Exception { - try (Connection conn = DriverManager.getConnection(getJdbcUrl())) { - String parentTableName = SchemaUtil.getTableName(generateUniqueName(), - generateUniqueName()); - String parentViewName = SchemaUtil.getTableName(generateUniqueName(), - generateUniqueName()); - String childViewName = SchemaUtil.getTableName(generateUniqueName(), - generateUniqueName()); - // create parent table - String ddl = "CREATE TABLE " + parentTableName - + " (col1 INTEGER NOT NULL, col2 INTEGER, col3 VARCHAR " - + "CONSTRAINT pk PRIMARY KEY (col1))"; - conn.createStatement().execute(ddl); - - // create view from table - ddl = "CREATE VIEW " + parentViewName + " AS SELECT * FROM " + parentTableName; - conn.createStatement().execute(ddl); - try { - ddl = "ALTER TABLE " + parentTableName + " DROP COLUMN col2"; - conn.createStatement().execute(ddl); - fail("ALTER TABLE DROP COLUMN should not be allowed on parent table"); - } catch (SQLException e) { - assertEquals(SQLExceptionCode.CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode()); - } - - // create child view from above view - ddl = "CREATE VIEW " + childViewName + "(col5 INTEGER) AS SELECT * FROM " - + parentViewName; - conn.createStatement().execute(ddl); - try { - ddl = "ALTER VIEW " + parentViewName + " DROP COLUMN col2"; - conn.createStatement().execute(ddl); - fail("ALTER VIEW DROP COLUMN should not be allowed on parent view"); - } catch (SQLException e) { - assertEquals(SQLExceptionCode.CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode()); - } - - // alter child view with drop column should be allowed - ddl = "ALTER VIEW " + childViewName + " DROP COLUMN col2"; - conn.createStatement().execute(ddl); - } - } -} \ No newline at end of file diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogIT.java deleted file mode 100644 index b55712f..0000000 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogIT.java +++ /dev/null @@ -1,140 +0,0 @@ -/* - * 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.fail; - -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.SQLException; -import java.sql.Statement; -import java.util.Collections; -import java.util.Map; -import java.util.Properties; - -import org.apache.hadoop.hbase.DoNotRetryIOException; -import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.client.RegionLocator; -import org.apache.phoenix.exception.SQLExceptionCode; -import org.apache.phoenix.query.BaseTest; -import org.apache.phoenix.query.QueryServices; -import org.apache.phoenix.util.PhoenixRuntime; -import org.apache.phoenix.util.ReadOnlyProps; -import org.apache.phoenix.util.SchemaUtil; -import org.junit.BeforeClass; -import org.junit.Test; -import org.junit.experimental.categories.Category; - -import com.google.common.collect.Maps; - -@Category(NeedsOwnMiniClusterTest.class) -public class SystemCatalogIT extends BaseTest { - private HBaseTestingUtility testUtil = null; - - @BeforeClass - public static synchronized void doSetup() throws Exception { - Map<String, String> serverProps = Maps.newHashMapWithExpectedSize(1); - serverProps.put(QueryServices.SYSTEM_CATALOG_SPLITTABLE, "false"); - serverProps.put(QueryServices.ALLOW_SPLITTABLE_SYSTEM_CATALOG_ROLLBACK, "true"); - Map<String, String> clientProps = Collections.emptyMap(); - setUpTestDriver(new ReadOnlyProps(serverProps.entrySet().iterator()), - new ReadOnlyProps(clientProps.entrySet().iterator())); - } - - /** - * Make sure that SYSTEM.CATALOG cannot be split if QueryServices.SYSTEM_CATALOG_SPLITTABLE is false - */ - @Test - public void testSystemTableSplit() throws Exception { - testUtil = getUtility(); - for (int i=0; i<10; i++) { - createTable("schema"+i+".table_"+i); - } - TableName systemCatalog = TableName.valueOf("SYSTEM.CATALOG"); - RegionLocator rl = testUtil.getConnection().getRegionLocator(systemCatalog); - assertEquals(rl.getAllRegionLocations().size(), 1); - try { - // now attempt to split SYSTEM.CATALOG - testUtil.getHBaseAdmin().split(systemCatalog); - // make sure the split finishes (there's no synchronous splitting before HBase 2.x) - testUtil.getHBaseAdmin().disableTable(systemCatalog); - testUtil.getHBaseAdmin().enableTable(systemCatalog); - } catch (DoNotRetryIOException e) { - // table is not splittable - assert (e.getMessage().contains("NOT splittable")); - } - - // test again... Must still be exactly one region. - rl = testUtil.getConnection().getRegionLocator(systemCatalog); - assertEquals(1, rl.getAllRegionLocations().size()); - } - - private void createTable(String tableName) throws Exception { - try (Connection conn = DriverManager.getConnection(getJdbcUrl()); - Statement stmt = conn.createStatement();) { - stmt.execute("DROP TABLE IF EXISTS " + tableName); - stmt.execute("CREATE TABLE " + tableName - + " (TENANT_ID VARCHAR NOT NULL, PK1 VARCHAR NOT NULL, V1 VARCHAR CONSTRAINT PK " + - "PRIMARY KEY(TENANT_ID, PK1)) MULTI_TENANT=true"); - try (Connection tenant1Conn = getTenantConnection("tenant1")) { - String view1DDL = "CREATE VIEW " + tableName + "_view AS SELECT * FROM " + tableName; - tenant1Conn.createStatement().execute(view1DDL); - } - conn.commit(); - } - } - - private String getJdbcUrl() { - return "jdbc:phoenix:localhost:" + getUtility().getZkCluster().getClientPort() + ":/hbase"; - } - - private Connection getTenantConnection(String tenantId) throws SQLException { - Properties tenantProps = new Properties(); - tenantProps.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId); - return DriverManager.getConnection(getJdbcUrl(), tenantProps); - } - - /** - * Ensure that we cannot add a column to a base table if QueryServices.BLOCK_METADATA_CHANGES_REQUIRE_PROPAGATION - * is true - */ - @Test - public void testAddingColumnFails() throws Exception { - try (Connection conn = DriverManager.getConnection(getJdbcUrl())) { - String fullTableName = SchemaUtil.getTableName(generateUniqueName(), generateUniqueName()); - String fullViewName = SchemaUtil.getTableName(generateUniqueName(), generateUniqueName()); - String ddl = "CREATE TABLE " + fullTableName + " (k1 INTEGER NOT NULL, v1 INTEGER " + - "CONSTRAINT pk PRIMARY KEY (k1))"; - conn.createStatement().execute(ddl); - - ddl = "CREATE VIEW " + fullViewName + " AS SELECT * FROM " + fullTableName; - conn.createStatement().execute(ddl); - - try { - ddl = "ALTER TABLE " + fullTableName + " ADD v2 INTEGER"; - conn.createStatement().execute(ddl); - fail(); - } - catch (SQLException e) { - assertEquals(SQLExceptionCode.CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode()); - } - } - } -} \ No newline at end of file diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogRollbackEnabledIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogRollbackEnabledIT.java new file mode 100644 index 0000000..2b69596 --- /dev/null +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogRollbackEnabledIT.java @@ -0,0 +1,298 @@ +/* + * 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 static org.junit.Assert.fail; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + +import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.RegionLocator; +import org.apache.phoenix.exception.SQLExceptionCode; +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData; +import org.apache.phoenix.query.BaseTest; +import org.apache.phoenix.query.QueryServices; +import org.apache.phoenix.schema.ColumnNotFoundException; +import org.apache.phoenix.util.PhoenixRuntime; +import org.apache.phoenix.util.ReadOnlyProps; +import org.apache.phoenix.util.SchemaUtil; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +/** + * Tests various scenarios when + * {@link QueryServices#ALLOW_SPLITTABLE_SYSTEM_CATALOG_ROLLBACK} + * is set to true and SYSTEM.CATALOG should not be allowed to split. + * Note that this config must + * be set on both the client and server + */ +@Category(NeedsOwnMiniClusterTest.class) +public class SystemCatalogRollbackEnabledIT extends BaseTest { + + @BeforeClass + public static synchronized void doSetup() throws Exception { + Map<String, String> serverProps = new HashMap<>(2); + Map<String, String> clientProps = new HashMap<>(1); + serverProps.put(QueryServices.SYSTEM_CATALOG_SPLITTABLE, "false"); + serverProps.put(QueryServices.ALLOW_SPLITTABLE_SYSTEM_CATALOG_ROLLBACK, + "true"); + clientProps.put(QueryServices.ALLOW_SPLITTABLE_SYSTEM_CATALOG_ROLLBACK, + "true"); + setUpTestDriver(new ReadOnlyProps(serverProps.entrySet().iterator()), + new ReadOnlyProps(clientProps.entrySet().iterator())); + } + + private void createTableAndTenantViews(String tableName) throws Exception { + try (Connection conn = DriverManager.getConnection(getUrl()); + Statement stmt = conn.createStatement();) { + stmt.execute("DROP TABLE IF EXISTS " + tableName); + stmt.execute("CREATE TABLE " + tableName + + " (TENANT_ID VARCHAR NOT NULL, " + + "PK1 VARCHAR NOT NULL, V1 VARCHAR CONSTRAINT PK " + + "PRIMARY KEY(TENANT_ID, PK1)) MULTI_TENANT=true"); + try (Connection tenant1Conn = getTenantConnection("tenant1")) { + String view1DDL = "CREATE VIEW " + tableName + + "_view1 AS SELECT * FROM " + + tableName; + tenant1Conn.createStatement().execute(view1DDL); + } + try (Connection tenant2Conn = getTenantConnection("tenant2")) { + String view1DDL = "CREATE VIEW " + tableName + + "_view2 AS SELECT * FROM " + tableName; + tenant2Conn.createStatement().execute(view1DDL); + } + conn.commit(); + } + } + + private Connection getTenantConnection(String tenantId) + throws SQLException { + Properties tenantProps = new Properties(); + tenantProps.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId); + return DriverManager.getConnection(getUrl(), tenantProps); + } + + + /** + * Make sure that SYSTEM.CATALOG cannot be split if + * {@link QueryServices#SYSTEM_CATALOG_SPLITTABLE} is false + */ + @Test + public void testSystemTableDoesNotSplit() throws Exception { + HBaseTestingUtility testUtil = getUtility(); + for (int i=0; i<10; i++) { + createTableAndTenantViews("schema"+i+".table_"+i); + } + TableName systemCatalog = TableName.valueOf( + PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME); + RegionLocator rl = testUtil.getConnection() + .getRegionLocator(systemCatalog); + assertEquals(1, rl.getAllRegionLocations().size()); + try { + // now attempt to split SYSTEM.CATALOG + testUtil.getHBaseAdmin().split(systemCatalog); + // make sure the split finishes (there's no synchronous splitting + // before HBase 2.x) + testUtil.getHBaseAdmin().disableTable(systemCatalog); + testUtil.getHBaseAdmin().enableTable(systemCatalog); + } catch (DoNotRetryIOException e) { + // table is not splittable + assertTrue(e.getMessage().contains("NOT splittable")); + } + + // test again... Must still be exactly one region. + rl = testUtil.getConnection().getRegionLocator(systemCatalog); + assertEquals(1, rl.getAllRegionLocations().size()); + } + + /** + * Ensure that we cannot add a column to a parent table or view if + * {@link QueryServices#ALLOW_SPLITTABLE_SYSTEM_CATALOG_ROLLBACK} is true + */ + @Test + public void testAddColumnOnParentFails() throws Exception { + try (Connection conn = DriverManager.getConnection(getUrl())) { + final String parentTableName = SchemaUtil.getTableName( + generateUniqueName(), generateUniqueName()); + final String parentViewName = SchemaUtil.getTableName( + generateUniqueName(), generateUniqueName()); + final String childViewName = SchemaUtil.getTableName( + generateUniqueName(), generateUniqueName()); + // create parent table + String ddl = "CREATE TABLE " + parentTableName + + " (col1 INTEGER NOT NULL," + + " col2 INTEGER " + "CONSTRAINT pk PRIMARY KEY (col1))"; + conn.createStatement().execute(ddl); + + // create view on table + ddl = "CREATE VIEW " + parentViewName + " AS SELECT * FROM " + + parentTableName; + conn.createStatement().execute(ddl); + try { + ddl = "ALTER TABLE " + parentTableName + " ADD col4 INTEGER"; + conn.createStatement().execute(ddl); + fail("ALTER TABLE ADD should not be allowed on parent table"); + } catch (SQLException e) { + assertEquals(SQLExceptionCode.CANNOT_MUTATE_TABLE + .getErrorCode(), e.getErrorCode()); + } + + // create child view on above view + ddl = "CREATE VIEW " + childViewName + + "(col3 INTEGER) AS SELECT * FROM " + parentViewName; + conn.createStatement().execute(ddl); + try { + ddl = "ALTER VIEW " + parentViewName + " ADD col4 INTEGER"; + conn.createStatement().execute(ddl); + fail("ALTER VIEW ADD should not be allowed on parent view"); + } catch (SQLException e) { + assertEquals(SQLExceptionCode.CANNOT_MUTATE_TABLE + .getErrorCode(), e.getErrorCode()); + } + + // alter child view with add column should be allowed + ddl = "ALTER VIEW " + childViewName + " ADD col4 INTEGER"; + conn.createStatement().execute(ddl); + } + } + + /** + * Ensure that we cannot drop a column from a parent table or view if + * {@link QueryServices#ALLOW_SPLITTABLE_SYSTEM_CATALOG_ROLLBACK} is true + */ + @Test + public void testDropColumnOnParentFails() throws Exception { + try (Connection conn = DriverManager.getConnection(getUrl())) { + final String parentTableName = SchemaUtil.getTableName( + generateUniqueName(), generateUniqueName()); + final String parentViewName = SchemaUtil.getTableName( + generateUniqueName(), generateUniqueName()); + final String childViewName = SchemaUtil.getTableName( + generateUniqueName(), generateUniqueName()); + // create parent table + String ddl = "CREATE TABLE " + parentTableName + + " (col1 INTEGER NOT NULL, col2 INTEGER, col3 VARCHAR " + + "CONSTRAINT pk PRIMARY KEY (col1))"; + conn.createStatement().execute(ddl); + + // create view on table + ddl = "CREATE VIEW " + parentViewName + " AS SELECT * FROM " + + parentTableName; + conn.createStatement().execute(ddl); + try { + ddl = "ALTER TABLE " + parentTableName + " DROP COLUMN col2"; + conn.createStatement().execute(ddl); + fail("ALTER TABLE DROP COLUMN should not be allowed on " + + "parent table"); + } catch (SQLException e) { + assertEquals(SQLExceptionCode.CANNOT_MUTATE_TABLE + .getErrorCode(), e.getErrorCode()); + } + + // create child view on above view + ddl = "CREATE VIEW " + childViewName + + "(col5 INTEGER) AS SELECT * FROM " + parentViewName; + conn.createStatement().execute(ddl); + try { + ddl = "ALTER VIEW " + parentViewName + " DROP COLUMN col2"; + conn.createStatement().execute(ddl); + fail("ALTER VIEW DROP COLUMN should not be allowed on " + + "parent view"); + } catch (SQLException e) { + assertEquals(SQLExceptionCode.CANNOT_MUTATE_TABLE + .getErrorCode(), e.getErrorCode()); + } + + // alter child view with drop column should be allowed + ddl = "ALTER VIEW " + childViewName + " DROP COLUMN col2"; + conn.createStatement().execute(ddl); + } + } + + // Test for PHOENIX-6032 + @Test + public void testViewDoesNotSeeDataForDroppedColumn() throws Exception { + final String parentName = "T_" + SchemaUtil.getTableName( + generateUniqueName(), generateUniqueName()); + final String viewName = "V_" + SchemaUtil.getTableName( + generateUniqueName(), generateUniqueName()); + try (Connection conn = DriverManager.getConnection(getUrl())) { + conn.createStatement().execute("CREATE TABLE " + parentName + + " (A INTEGER PRIMARY KEY, B INTEGER, C" + + " VARCHAR, D INTEGER)"); + conn.createStatement().execute("CREATE VIEW " + viewName + + " (VA INTEGER, VB INTEGER)" + + " AS SELECT * FROM " + parentName + " WHERE B=200"); + // Upsert some data via the view + conn.createStatement().execute("UPSERT INTO " + viewName + + " (A,B,C,D,VA,VB) VALUES" + + " (2, 200, 'def', -20, 91, 101)"); + conn.commit(); + } + + try (Connection conn = DriverManager.getConnection(getUrl())) { + // Query from the parent table and assert expected values + ResultSet rs = conn.createStatement().executeQuery( + "SELECT A,B,C,D FROM " + parentName); + assertTrue(rs.next()); + assertEquals(2, rs.getInt(1)); + assertEquals(200, rs.getInt(2)); + assertEquals("def", rs.getString(3)); + assertEquals(-20, rs.getInt(4)); + assertFalse(rs.next()); + + // Query from the view and assert expected values + rs = conn.createStatement().executeQuery( + "SELECT A,B,C,D,VA,VB FROM " + viewName); + assertTrue(rs.next()); + assertEquals(2, rs.getInt(1)); + assertEquals(200, rs.getInt(2)); + assertEquals("def", rs.getString(3)); + assertEquals(-20, rs.getInt(4)); + assertEquals(91, rs.getInt(5)); + assertEquals(101, rs.getInt(6)); + assertFalse(rs.next()); + } + + try (Connection conn = DriverManager.getConnection(getUrl())) { + // Drop a parent column from the view + conn.createStatement().execute("ALTER VIEW " + viewName + + " DROP COLUMN C"); + try { + conn.createStatement().executeQuery("SELECT C FROM " + + viewName); + fail("Expected a ColumnNotFoundException for C since it " + + "was dropped"); + } catch (ColumnNotFoundException ignore) { } + } + } + +} \ No newline at end of file diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java index 659bf91..f4874ed 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java @@ -2648,11 +2648,9 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso } } if (!isMutationAllowed) { - MetaDataMutationResult metaDataMutationResult = - new MetaDataMutationResult( + return new MetaDataMutationResult( MetaDataProtocol.MutationCode.UNALLOWED_TABLE_MUTATION, EnvironmentEdgeManager.currentTimeMillis(), null); - return metaDataMutationResult; } if (!isSchemaMutationAllowed) { MetaDataMutationResult metaDataMutationResult = diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/ViewUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/ViewUtil.java index 5e4d535..0915da7 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/util/ViewUtil.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/util/ViewUtil.java @@ -773,7 +773,7 @@ public class ViewUtil { * If the same column is present in the parent and child * (for table metadata created before PHOENIX-3534 or when * {@link org.apache.phoenix.query.QueryServices#ALLOW_SPLITTABLE_SYSTEM_CATALOG_ROLLBACK} is - * enabled) we choose the child column over the parent column. + * enabled) we choose the latest column. * Note that we don't need to call this method for views created before 4.15 since they * already contain all the columns from their ancestors. * @param view PTable of the view @@ -788,6 +788,7 @@ public class ViewUtil { List<PColumn> currAncestorTableCols = PTableImpl.getColumnsToClone(parentTable); if (currAncestorTableCols != null) { // add the ancestor columns in reverse order so that the final column list + // (reversed outside of this method invocation) // contains ancestor columns and then the view columns in the right order for (int j = currAncestorTableCols.size() - 1; j >= 0; j--) { PColumn ancestorColumn = currAncestorTableCols.get(j); @@ -829,10 +830,23 @@ public class ViewUtil { if (!isDiverged && ancestorColumn.getTimestamp() > existingColumn.getTimestamp()) { allColumns.remove(existingColumnIndex); - allColumns.add(ancestorColumn); + // Remove the existing column and add the ancestor + // column at the end and make sure to mark it + // as derived + allColumns.add(new PColumnImpl(ancestorColumn, true, + ancestorColumn.getPosition())); + } else { + // Since this is a column from the ancestor, + // mark it as derived + allColumns.set(existingColumnIndex, + new PColumnImpl(existingColumn, true, + existingColumn.getPosition())); } } else { - allColumns.add(ancestorColumn); + // Since this is a column from the ancestor, + // mark it as derived + allColumns.add(new PColumnImpl(ancestorColumn, true, + ancestorColumn.getPosition())); } } }