You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by ta...@apache.org on 2019/02/25 07:57:23 UTC

[impala] 09/14: IMPALA-6305: Allow column definitions in ALTER VIEW

This is an automated email from the ASF dual-hosted git repository.

tarmstrong pushed a commit to branch 2.x
in repository https://gitbox.apache.org/repos/asf/impala.git

commit 1219d8fb11358c36284e459e0b9fd37d26f44df9
Author: poojanilangekar <po...@cloudera.com>
AuthorDate: Thu Jun 14 14:49:05 2018 -0700

    IMPALA-6305: Allow column definitions in ALTER VIEW
    
    This change adds support to change column definitions in ALTER VIEW
    statements. This support only required minor changes in the parser
    and the AlterViewStmt constructor.
    
    Here's an example syntax:
        alter view foo (a, b comment 'helloworld') as
        select * from bar;
    
        describe foo;
        +------+--------+------------+
        | name | type   | comment    |
        +------+--------+------------+
        | a    | string |            |
        | b    | string | helloworld |
        +------+--------+------------+
    
    The following tests were modified:
    1. ParserTest - To check that the parser handles column definitions
       for alter view statements.
    2. AnalyzerDDLTest - To ensure that the analyzer supports the
       change column definitions parsed.
    3. TestDdlStatements - To verify the end-to-end functioning of
       ALTER VIEW statements with change column definitions.
    4. AuthorizationTest - To ensure that alter table commands with
       column definitions check permissions as expected.
    
    Change-Id: I6073444a814a24d97e80df15fcd39be2812f63fc
    Reviewed-on: http://gerrit.cloudera.org:8080/10720
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 fe/src/main/cup/sql-parser.cup                     |  5 +-
 .../org/apache/impala/analysis/AlterViewStmt.java  | 14 ++++--
 .../impala/analysis/CreateOrAlterViewStmtBase.java |  6 +--
 .../org/apache/impala/analysis/CreateViewStmt.java |  5 +-
 .../org/apache/impala/analysis/AnalyzeDDLTest.java | 37 +++++++++++++++
 .../apache/impala/analysis/AuthorizationTest.java  | 23 +++++++++
 .../org/apache/impala/analysis/ParserTest.java     | 15 ++++++
 .../queries/QueryTest/views-ddl.test               | 54 ++++++++++++++++++++++
 8 files changed, 148 insertions(+), 11 deletions(-)

diff --git a/fe/src/main/cup/sql-parser.cup b/fe/src/main/cup/sql-parser.cup
index db69aae..c2ded20 100644
--- a/fe/src/main/cup/sql-parser.cup
+++ b/fe/src/main/cup/sql-parser.cup
@@ -1869,8 +1869,9 @@ view_column_def ::=
   ;
 
 alter_view_stmt ::=
-  KW_ALTER KW_VIEW table_name:table KW_AS query_stmt:view_def
-  {: RESULT = new AlterViewStmt(table, view_def); :}
+  KW_ALTER KW_VIEW table_name:table view_column_defs:col_defs KW_AS
+  query_stmt:view_def
+  {: RESULT = new AlterViewStmt(table, col_defs, view_def); :}
   | KW_ALTER KW_VIEW table_name:before_table KW_RENAME KW_TO table_name:new_table
   {: RESULT = new AlterTableOrViewRenameStmt(before_table, new_table, false); :}
   ;
diff --git a/fe/src/main/java/org/apache/impala/analysis/AlterViewStmt.java b/fe/src/main/java/org/apache/impala/analysis/AlterViewStmt.java
index a6b6548..cad0169 100644
--- a/fe/src/main/java/org/apache/impala/analysis/AlterViewStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/AlterViewStmt.java
@@ -17,22 +17,27 @@
 
 package org.apache.impala.analysis;
 
+import java.util.List;
+
 import org.apache.impala.authorization.Privilege;
 import org.apache.impala.catalog.FeTable;
 import org.apache.impala.catalog.FeView;
 import org.apache.impala.common.AnalysisException;
 import org.apache.impala.common.RuntimeEnv;
+import org.apache.impala.thrift.TAccessEvent;
+import org.apache.impala.thrift.TCatalogObjectType;
 import org.apache.impala.service.BackendConfig;
 
+import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 
 /**
  * Represents an ALTER VIEW AS statement.
  */
 public class AlterViewStmt extends CreateOrAlterViewStmtBase {
-
-  public AlterViewStmt(TableName tableName, QueryStmt viewDefStmt) {
-    super(false, tableName, null, null, viewDefStmt);
+  public AlterViewStmt(
+      TableName tableName, List<ColumnDef> columnDefs, QueryStmt viewDefStmt) {
+    super(false, tableName, columnDefs, null, viewDefStmt);
   }
 
   @Override
@@ -51,6 +56,8 @@ public class AlterViewStmt extends CreateOrAlterViewStmtBase {
       throw new AnalysisException(String.format(
           "ALTER VIEW not allowed on a table: %s.%s", dbName_, getTbl()));
     }
+    analyzer.addAccessEvent(new TAccessEvent(dbName_ + "." + tableName_.getTbl(),
+        TCatalogObjectType.VIEW, Privilege.ALTER.toString()));
 
     createColumnAndViewDefs(analyzer);
     if (BackendConfig.INSTANCE.getComputeLineage() || RuntimeEnv.INSTANCE.isTestEnv()) {
@@ -66,6 +73,7 @@ public class AlterViewStmt extends CreateOrAlterViewStmtBase {
       sb.append(tableName_.getDb() + ".");
     }
     sb.append(tableName_.getTbl());
+    if (columnDefs_ != null) sb.append("(" + Joiner.on(", ").join(columnDefs_) + ")");
     sb.append(" AS " + viewDefStmt_.toSql());
     return sb.toString();
   }
diff --git a/fe/src/main/java/org/apache/impala/analysis/CreateOrAlterViewStmtBase.java b/fe/src/main/java/org/apache/impala/analysis/CreateOrAlterViewStmtBase.java
index 4310871..6cdec98 100644
--- a/fe/src/main/java/org/apache/impala/analysis/CreateOrAlterViewStmtBase.java
+++ b/fe/src/main/java/org/apache/impala/analysis/CreateOrAlterViewStmtBase.java
@@ -40,7 +40,7 @@ public abstract class CreateOrAlterViewStmtBase extends StatementBase {
 
   protected final boolean ifNotExists_;
   protected final TableName tableName_;
-  protected final ArrayList<ColumnDef> columnDefs_;
+  protected final List<ColumnDef> columnDefs_;
   protected final String comment_;
   protected final QueryStmt viewDefStmt_;
 
@@ -67,10 +67,10 @@ public abstract class CreateOrAlterViewStmtBase extends StatementBase {
 
   // Columns to use in the select list of the expanded SQL string and when registering
   // this view in the metastore. Set in analysis.
-  protected ArrayList<ColumnDef> finalColDefs_;
+  protected List<ColumnDef> finalColDefs_;
 
   public CreateOrAlterViewStmtBase(boolean ifNotExists, TableName tableName,
-      ArrayList<ColumnDef> columnDefs, String comment, QueryStmt viewDefStmt) {
+      List<ColumnDef> columnDefs, String comment, QueryStmt viewDefStmt) {
     Preconditions.checkNotNull(tableName);
     Preconditions.checkNotNull(viewDefStmt);
     this.ifNotExists_ = ifNotExists;
diff --git a/fe/src/main/java/org/apache/impala/analysis/CreateViewStmt.java b/fe/src/main/java/org/apache/impala/analysis/CreateViewStmt.java
index 6e98fe5..1a17aa8 100644
--- a/fe/src/main/java/org/apache/impala/analysis/CreateViewStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/CreateViewStmt.java
@@ -17,7 +17,7 @@
 
 package org.apache.impala.analysis;
 
-import java.util.ArrayList;
+import java.util.List;
 
 import org.apache.impala.authorization.Privilege;
 import org.apache.impala.common.AnalysisException;
@@ -33,9 +33,8 @@ import com.google.common.base.Preconditions;
  * Represents a CREATE VIEW statement.
  */
 public class CreateViewStmt extends CreateOrAlterViewStmtBase {
-
   public CreateViewStmt(boolean ifNotExists, TableName tableName,
-      ArrayList<ColumnDef> columnDefs, String comment, QueryStmt viewDefStmt) {
+      List<ColumnDef> columnDefs, String comment, QueryStmt viewDefStmt) {
     super(ifNotExists, tableName, columnDefs, comment, viewDefStmt);
   }
 
diff --git a/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java b/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
index 634fb87..550ab24 100644
--- a/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
@@ -1087,11 +1087,30 @@ public class AnalyzeDDLTest extends FrontendTestBase {
     // View-definition references a view.
     AnalyzesOk("alter view functional.alltypes_view as " +
         "select * from functional.alltypes_view");
+    // Change column definitions.
+    AnalyzesOk("alter view functional.alltypes_view (a, b) as " +
+        "select int_col, string_col from functional.alltypes");
+    // Change column definitions after renaming columns in select.
+    AnalyzesOk("alter view functional.alltypes_view (a, b) as " +
+        "select int_col x, string_col y from functional.alltypes");
 
     // View-definition resulting in Hive-style auto-generated column names.
     AnalyzesOk("alter view functional.alltypes_view as " +
         "select trim('abc'), 17 * 7");
 
+    // Altering a view on a view is ok (alltypes_view is a view on alltypes).
+    AnalyzesOk("alter view functional.alltypes_view (aaa, bbb) as " +
+        "select * from functional.complex_view");
+
+    // Altering a view with same column as existing one.
+    AnalyzesOk("alter view functional.complex_view (abc, xyz) as " +
+        "select year, month from functional.alltypes_view");
+
+    // Alter view with joins and aggregates.
+    AnalyzesOk("alter view functional.alltypes_view (cnt) as " +
+        "select count(distinct x.int_col) from functional.alltypessmall x " +
+        "inner join functional.alltypessmall y on (x.id = y.id) group by x.bigint_col");
+
     // Cannot ALTER VIEW a table.
     AnalysisError("alter view functional.alltypes as " +
         "select * from functional.alltypesagg",
@@ -1127,6 +1146,24 @@ public class AnalyzeDDLTest extends FrontendTestBase {
     AnalyzesOk("alter view functional.alltypes_view as " +
         "select * from functional.alltypestiny where id in " +
         "(select id from functional.alltypessmall where int_col = 1)");
+    // Mismatching number of columns in column definition and view-alteration statement.
+    AnalysisError("alter view functional.alltypes_view (a) as " +
+        "select int_col, string_col from functional.alltypes",
+        "Column-definition list has fewer columns (1) than the " +
+        "view-definition query statement returns (2).");
+    AnalysisError("alter view functional.alltypes_view (a, b, c) as " +
+        "select int_col from functional.alltypes",
+        "Column-definition list has more columns (3) than the " +
+        "view-definition query statement returns (1).");
+    // Duplicate columns in the view-alteration statement.
+    AnalysisError("alter view functional.alltypes_view as " +
+        "select * from functional.alltypessmall a " +
+        "inner join functional.alltypessmall b on a.id = b.id",
+        "Duplicate column name: id");
+    // Duplicate columns in the column definition.
+    AnalysisError("alter view functional.alltypes_view (a, b, a) as " +
+        "select int_col, int_col, int_col from functional.alltypes",
+        "Duplicate column name: a");
   }
 
   @Test
diff --git a/fe/src/test/java/org/apache/impala/analysis/AuthorizationTest.java b/fe/src/test/java/org/apache/impala/analysis/AuthorizationTest.java
index 36bcaa3..a684847 100644
--- a/fe/src/test/java/org/apache/impala/analysis/AuthorizationTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/AuthorizationTest.java
@@ -1656,6 +1656,8 @@ public class AuthorizationTest extends FrontendTestBase {
   public void TestAlterView() throws ImpalaException {
     AuthzOk("ALTER VIEW functional_seq_snap.alltypes_view rename to " +
         "functional_seq_snap.v1");
+    AuthzOk("ALTER VIEW functional.alltypes_view (a, b, c) as " +
+        "select int_col, string_col, timestamp_col from functional.alltypesagg");
 
     // ALTER privilege on view only. RENAME also requires CREATE privileges on the DB.
     AuthzOk("ALTER VIEW functional.alltypes_view rename to functional_seq_snap.view_view_1");
@@ -1682,6 +1684,10 @@ public class AuthorizationTest extends FrontendTestBase {
         "functional_seq_snap.new_view",
         "User '%s' does not have privileges to execute 'ALTER' on: " +
         "functional.alltypes");
+    AuthzError("ALTER VIEW functional.alltypes (a, b, c) as " +
+        "select int_col, string_col, timestamp_col from functional.alltypesagg",
+        "User '%s' does not have privileges to execute 'ALTER' on: " +
+        "functional.alltypes");
 
     // Rename view that does not exist (no permissions).
     AuthzError("ALTER VIEW functional.notbl rename to functional_seq_snap.newtbl",
@@ -1694,26 +1700,43 @@ public class AuthorizationTest extends FrontendTestBase {
     // Alter view that does not exist (no permissions).
     AuthzError("ALTER VIEW functional.notbl rename to functional_seq_snap.new_view",
         "User '%s' does not have privileges to execute 'ALTER' on: functional.notbl");
+    AuthzError("ALTER VIEW functional.notbl (a, b, c) as " +
+        "select int_col, string_col, timestamp_col from functional.alltypesagg",
+        "User '%s' does not have privileges to execute 'ALTER' on: functional.notbl");
 
     // Alter view in db that does not exist (no permissions).
     AuthzError("ALTER VIEW nodb.alltypes rename to functional_seq_snap.new_view",
         "User '%s' does not have privileges to execute 'ALTER' on: nodb.alltypes");
+    AuthzError("ALTER VIEW nodb.alltypes (a, b, c) as " +
+        "select int_col, string_col, timestamp_col from functional.alltypesagg",
+        "User '%s' does not have privileges to execute 'ALTER' on: nodb.alltypes");
 
     // Unqualified view name.
     AuthzError("ALTER VIEW alltypes rename to functional_seq_snap.new_view",
         "User '%s' does not have privileges to execute 'ALTER' on: default.alltypes");
+    AuthzError("ALTER VIEW alltypes (a, b, c) as " +
+        "select int_col, string_col, timestamp_col from functional.alltypesagg",
+        "User '%s' does not have privileges to execute 'ALTER' on: default.alltypes");
 
     // No permissions on target view.
     AuthzError("alter view functional.alltypes_view_sub as " +
         "select * from functional.alltypesagg",
         "User '%s' does not have privileges to execute 'ALTER' on: " +
         "functional.alltypes_view");
+    AuthzError("alter view functional.alltypes_view_sub (a, b, c) as " +
+        "select int_col, string_col, timestamp_col from functional.alltypesagg",
+        "User '%s' does not have privileges to execute 'ALTER' on: " +
+        "functional.alltypes_view");
 
     // No permissions on source view.
     AuthzError("alter view functional_seq_snap.alltypes_view " +
         "as select * from functional.alltypes_view",
         "User '%s' does not have privileges to execute 'SELECT' on: " +
         "functional.alltypes_view");
+    AuthzError("alter view functional_seq_snap.alltypes_view (a, b, c) " +
+        "as select int_col, string_col, timestamp_col from functional.alltypes_view",
+        "User '%s' does not have privileges to execute 'SELECT' on: " +
+        "functional.alltypes_view");
   }
 
   @Test
diff --git a/fe/src/test/java/org/apache/impala/analysis/ParserTest.java b/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
index 93c2e2b..1f19823 100644
--- a/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
@@ -2906,12 +2906,19 @@ public class ParserTest extends FrontendTestBase {
     ParsesOk("ALTER VIEW Bar AS SELECT a, b, c FROM t");
     ParsesOk("ALTER VIEW Bar AS VALUES(1, 2, 3)");
     ParsesOk("ALTER VIEW Bar AS SELECT 1, 2, 3 UNION ALL select 4, 5, 6");
+    ParsesOk("ALTER VIEW Bar (x, y, z) AS SELECT a, b, c from t");
+    ParsesOk("ALTER VIEW Bar (x, y COMMENT 'foo', z) AS SELECT a, b, c from t");
 
     ParsesOk("ALTER VIEW Foo.Bar AS SELECT 1, 2, 3");
     ParsesOk("ALTER VIEW Foo.Bar AS SELECT a, b, c FROM t");
     ParsesOk("ALTER VIEW Foo.Bar AS VALUES(1, 2, 3)");
     ParsesOk("ALTER VIEW Foo.Bar AS SELECT 1, 2, 3 UNION ALL select 4, 5, 6");
     ParsesOk("ALTER VIEW Foo.Bar AS WITH t AS (SELECT 1, 2, 3) SELECT * FROM t");
+    ParsesOk("ALTER VIEW Foo.Bar (x, y, z) AS SELECT a, b, c from t");
+    ParsesOk("ALTER VIEW Foo.Bar (x, y, z COMMENT 'foo') AS SELECT a, b, c from t");
+
+    // Mismatched number of columns in column definition and view definition parses ok.
+    ParsesOk("ALTER VIEW Bar (x, y) AS SELECT 1, 2, 3");
 
     // Must be ALTER VIEW not ALTER TABLE.
     ParserError("ALTER TABLE Foo.Bar AS SELECT 1, 2, 3");
@@ -2921,6 +2928,14 @@ public class ParserTest extends FrontendTestBase {
     ParserError("ALTER VIEW Foo.Bar SELECT 1, 2, 3");
     // Missing view definition.
     ParserError("ALTER VIEW Foo.Bar AS");
+    // Empty column definition not allowed.
+    ParserError("ALTER VIEW Foo.Bar () AS SELECT c FROM t");
+    // Column definitions cannot include types.
+    ParserError("ALTER VIEW Foo.Bar (x int) AS SELECT c FROM t");
+    ParserError("ALTER VIEW Foo.Bar (x int COMMENT 'x') AS SELECT c FROM t");
+    // A type does not parse as an identifier.
+    ParserError("ALTER VIEW Foo.Bar (int COMMENT 'x') AS SELECT c FROM t");
+
     // Invalid view definitions. A view definition must be a query statement.
     ParserError("ALTER VIEW Foo.Bar AS INSERT INTO t select * from t");
     ParserError("ALTER VIEW Foo.Bar AS UPSERT INTO t select * from t");
diff --git a/testdata/workloads/functional-query/queries/QueryTest/views-ddl.test b/testdata/workloads/functional-query/queries/QueryTest/views-ddl.test
index 0461222..ae83e34 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/views-ddl.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/views-ddl.test
@@ -230,6 +230,60 @@ show tables in $DATABASE
 'view_on_view'
 ====
 ---- QUERY
+# Alter a view with predicates, joins, aggregates and order by
+alter view $DATABASE.complex_view (aaa comment 'abc', bbb comment 'xyz') as
+select count(a.bigint_col), b.string_col from
+functional.alltypesagg a inner join functional.alltypestiny b
+on a.id = b.id where a.bigint_col < 50
+group by b.string_col having count(a.bigint_col) > 1
+order by b.string_col limit 100
+---- RESULTS
+'View has been altered.'
+====
+---- QUERY
+# Test verifying the description of the altered view
+describe $DATABASE.complex_view
+---- RESULTS
+'aaa','bigint','abc'
+'bbb','string','xyz'
+---- TYPES
+string,string,string
+====
+---- QUERY
+# Test querying the altered view
+select * from $DATABASE.complex_view;
+---- RESULTS
+2,'0'
+2,'1'
+---- TYPES
+bigint,string
+====
+---- QUERY
+# Alter a view on a view
+alter view $DATABASE.view_on_view (foo, bar) as
+select * from $DATABASE.complex_view
+---- RESULTS
+'View has been altered.'
+====
+---- QUERY
+# Test describing the description of the altered view
+describe $DATABASE.view_on_view
+---- RESULTS
+'foo','bigint',''
+'bar','string',''
+---- TYPES
+string,string,string
+====
+---- QUERY
+# Test querying the altered view
+select foo, bar from $DATABASE.view_on_view;
+---- RESULTS
+2,'0'
+2,'1'
+---- TYPES
+bigint,string
+====
+---- QUERY
 # Test altering a with a new definition
 alter view $DATABASE.new_parquet_view as
 select bigint_col, string_col from functional_parquet.alltypesagg