You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by tm...@apache.org on 2018/07/05 20:50:18 UTC

[1/2] impala git commit: IMPALA-6918: Implement COMMENT ON COLUMN

Repository: impala
Updated Branches:
  refs/heads/master 61e6a4777 -> fd0ba0fd2


IMPALA-6918: Implement COMMENT ON COLUMN

This patch implements updating comment on column table or view.

Syntax:
-- Add a comment 'comment' into column: db.table_or_view.col
COMMENT ON COLUMN db.table_or_view.col IS 'comment'

-- Delete a comment from column: db.table_or_view.col
COMMENT ON COLUMN db.table_or_view.col IS NULL

Testing:
- Added new front-end tests
- Ran all front-end tests
- Added new end-to-end tests
- Ran end-to-end DDL tests

Change-Id: I8976705b27e98c1540941d0a6cba0e18bad48437
Reviewed-on: http://gerrit.cloudera.org:8080/10754
Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/master
Commit: 4c5029afe76cf6ddff0a6a18322e386751065671
Parents: 61e6a47
Author: Fredy Wijaya <fw...@cloudera.com>
Authored: Mon Jun 18 22:48:56 2018 -0700
Committer: Impala Public Jenkins <im...@cloudera.com>
Committed: Thu Jul 5 19:08:03 2018 +0000

----------------------------------------------------------------------
 common/thrift/JniCatalog.thrift                 | 11 +++
 fe/src/main/cup/sql-parser.cup                  | 10 +++
 .../org/apache/impala/analysis/ColumnName.java  | 39 +++++++++
 .../impala/analysis/CommentOnColumnStmt.java    | 82 +++++++++++++++++++
 .../apache/impala/analysis/InlineViewRef.java   |  2 +
 .../impala/service/CatalogOpExecutor.java       | 85 ++++++++++++++++----
 .../apache/impala/analysis/AnalyzeDDLTest.java  | 21 ++++-
 .../impala/analysis/AuthorizationStmtTest.java  | 32 ++++++++
 .../org/apache/impala/analysis/ParserTest.java  |  9 +++
 tests/metadata/test_ddl.py                      | 75 ++++++++++++++---
 tests/metadata/test_ddl_base.py                 | 11 ++-
 11 files changed, 346 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/4c5029af/common/thrift/JniCatalog.thrift
----------------------------------------------------------------------
diff --git a/common/thrift/JniCatalog.thrift b/common/thrift/JniCatalog.thrift
index 7052faf..1876138 100644
--- a/common/thrift/JniCatalog.thrift
+++ b/common/thrift/JniCatalog.thrift
@@ -662,6 +662,14 @@ struct TGetCatalogUsageResponse{
   2: required list<TTableUsageMetrics> frequently_accessed_tables
 }
 
+struct TColumnName {
+  // Name of table/view.
+  1: required CatalogObjects.TTableName table_name
+
+  // Name of column.
+  2: required string column_name
+}
+
 struct TCommentOnParams {
   // Contents of comment to alter. When this field is not set, the comment will be removed.
   1: optional string comment
@@ -675,4 +683,7 @@ struct TCommentOnParams {
 
   // Name of table/view to alter.
   3: optional CatalogObjects.TTableName table_name
+
+  // Name of column to alter.
+  4: optional TColumnName column_name
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/4c5029af/fe/src/main/cup/sql-parser.cup
----------------------------------------------------------------------
diff --git a/fe/src/main/cup/sql-parser.cup b/fe/src/main/cup/sql-parser.cup
index c2ded20..bd00ad4 100644
--- a/fe/src/main/cup/sql-parser.cup
+++ b/fe/src/main/cup/sql-parser.cup
@@ -356,6 +356,7 @@ nonterminal String alias_clause;
 nonterminal ArrayList<String> ident_list, primary_keys;
 nonterminal ArrayList<String> opt_ident_list, opt_sort_cols;
 nonterminal TableName table_name;
+nonterminal ColumnName column_name;
 nonterminal FunctionName function_name;
 nonterminal Expr where_clause;
 nonterminal Expr predicate, bool_test_expr;
@@ -1034,6 +1035,8 @@ comment_on_stmt ::=
   {: RESULT = new CommentOnTableStmt(table, comment); :}
   | KW_COMMENT KW_ON KW_VIEW table_name:table KW_IS nullable_comment_val:comment
   {: RESULT = new CommentOnViewStmt(table, comment); :}
+  | KW_COMMENT KW_ON KW_COLUMN column_name:column KW_IS nullable_comment_val:comment
+  {: RESULT = new CommentOnColumnStmt(column, comment); :}
   ;
 
 // Introducing OWNER and USER keywords has a potential to be a breaking change,
@@ -2514,6 +2517,13 @@ table_name ::=
   {: RESULT = new TableName(db, tbl); :}
   ;
 
+column_name ::=
+  | ident_or_default:tbl DOT ident_or_default:col
+  {: RESULT = new ColumnName(new TableName(null, tbl), col); :}
+  | ident_or_default:db DOT ident_or_default:tbl DOT ident_or_default:col
+  {: RESULT = new ColumnName(new TableName(db, tbl), col); :}
+  ;
+
 function_name ::=
   // Use 'dotted_path' to avoid a reduce/reduce with slot_ref.
   dotted_path:path

http://git-wip-us.apache.org/repos/asf/impala/blob/4c5029af/fe/src/main/java/org/apache/impala/analysis/ColumnName.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/ColumnName.java b/fe/src/main/java/org/apache/impala/analysis/ColumnName.java
new file mode 100644
index 0000000..8eb285b
--- /dev/null
+++ b/fe/src/main/java/org/apache/impala/analysis/ColumnName.java
@@ -0,0 +1,39 @@
+// 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.impala.analysis;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Represents a column name that optionally includes its database name.
+ */
+public class ColumnName {
+  private final TableName tableName_;
+  private final String columnName_;
+
+  public ColumnName(TableName tableName, String columnName) {
+    Preconditions.checkNotNull(tableName);
+    Preconditions.checkNotNull(columnName);
+    this.tableName_ = tableName;
+    this.columnName_ = columnName;
+  }
+
+  public TableName getTableName() { return tableName_; }
+
+  public String getColumnName() { return columnName_; }
+}

http://git-wip-us.apache.org/repos/asf/impala/blob/4c5029af/fe/src/main/java/org/apache/impala/analysis/CommentOnColumnStmt.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/CommentOnColumnStmt.java b/fe/src/main/java/org/apache/impala/analysis/CommentOnColumnStmt.java
new file mode 100644
index 0000000..43e0eed
--- /dev/null
+++ b/fe/src/main/java/org/apache/impala/analysis/CommentOnColumnStmt.java
@@ -0,0 +1,82 @@
+// 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.impala.analysis;
+
+import com.google.common.base.Preconditions;
+import org.apache.impala.authorization.Privilege;
+import org.apache.impala.catalog.Column;
+import org.apache.impala.catalog.FeTable;
+import org.apache.impala.common.AnalysisException;
+import org.apache.impala.thrift.TColumnName;
+import org.apache.impala.thrift.TCommentOnParams;
+
+import java.util.List;
+
+/**
+ * Represents a COMMENT ON COLUMN table_or_view.col IS 'comment' statement.
+ */
+public class CommentOnColumnStmt extends CommentOnStmt {
+  private TableName tableName_;
+  private final String columnName_;
+
+  public CommentOnColumnStmt(ColumnName columnName, String comment) {
+    super(comment);
+    Preconditions.checkNotNull(columnName);
+    this.tableName_ = columnName.getTableName();
+    this.columnName_ = columnName.getColumnName();
+  }
+
+  @Override
+  public void collectTableRefs(List<TableRef> tblRefs) {
+    tblRefs.add(new TableRef(tableName_.toPath(), null));
+  }
+
+  @Override
+  public void analyze(Analyzer analyzer) throws AnalysisException {
+    super.analyze(analyzer);
+    tableName_ = analyzer.getFqTableName(tableName_);
+    // Although it makes sense to use column-level privilege, column-level privilege is
+    // only supported on tables and not views.
+    TableRef tableRef = new TableRef(tableName_.toPath(), null, Privilege.ALTER);
+    tableRef = analyzer.resolveTableRef(tableRef);
+    Preconditions.checkNotNull(tableRef);
+    tableRef.analyze(analyzer);
+    FeTable feTable;
+    String tableRefType = "";
+    if (tableRef instanceof InlineViewRef) {
+      InlineViewRef inlineViewRef = (InlineViewRef) tableRef;
+      feTable = inlineViewRef.getView();
+      tableRefType = "view";
+    } else {
+      feTable = tableRef.getTable();
+      tableRefType = "table";
+    }
+    Column column = feTable.getColumn(columnName_);
+    if (column == null) {
+      throw new AnalysisException(String.format(
+          "Column '%s' does not exist in %s: %s", columnName_, tableRefType, tableName_));
+    }
+  }
+
+  @Override
+  public TCommentOnParams toThrift() {
+    TCommentOnParams params = super.toThrift();
+    params.setColumn_name(new TColumnName(tableName_.toThrift(), columnName_));
+    return params;
+  }
+}

http://git-wip-us.apache.org/repos/asf/impala/blob/4c5029af/fe/src/main/java/org/apache/impala/analysis/InlineViewRef.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/InlineViewRef.java b/fe/src/main/java/org/apache/impala/analysis/InlineViewRef.java
index b1996bb..4fee2fa 100644
--- a/fe/src/main/java/org/apache/impala/analysis/InlineViewRef.java
+++ b/fe/src/main/java/org/apache/impala/analysis/InlineViewRef.java
@@ -307,6 +307,8 @@ public class InlineViewRef extends TableRef {
     return queryStmt_.getColLabels();
   }
 
+  public FeView getView() { return view_; }
+
   @Override
   protected TableRef clone() { return new InlineViewRef(this); }
 

http://git-wip-us.apache.org/repos/asf/impala/blob/4c5029af/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java b/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
index 03258e4..0ce5d29 100644
--- a/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
+++ b/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
@@ -51,6 +51,7 @@ import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.StringColumnStatsData;
 import org.apache.impala.analysis.AlterTableSortByStmt;
+import org.apache.impala.analysis.ColumnName;
 import org.apache.impala.analysis.FunctionName;
 import org.apache.impala.analysis.TableName;
 import org.apache.impala.authorization.User;
@@ -111,6 +112,7 @@ import org.apache.impala.thrift.TCatalogObject;
 import org.apache.impala.thrift.TCatalogObjectType;
 import org.apache.impala.thrift.TCatalogUpdateResult;
 import org.apache.impala.thrift.TColumn;
+import org.apache.impala.thrift.TColumnName;
 import org.apache.impala.thrift.TColumnStats;
 import org.apache.impala.thrift.TColumnType;
 import org.apache.impala.thrift.TColumnValue;
@@ -388,11 +390,7 @@ public class CatalogOpExecutor {
 
     TableName tableName = TableName.fromThrift(params.getTable_name());
     Table tbl = getExistingTable(tableName.getDb(), tableName.getTbl());
-
-    if (!catalog_.tryLockTable(tbl)) {
-      throw new InternalException(String.format("Error altering table %s due to lock " +
-          "contention.", tbl.getFullName()));
-    }
+    tryLock(tbl);
     final Timer.Context context
         = tbl.getMetrics().getTimer(Table.ALTER_DURATION_METRIC).time();
     try {
@@ -694,11 +692,7 @@ public class CatalogOpExecutor {
           "Null or empty column list given as argument to DdlExecutor.alterView");
     Table tbl = catalog_.getTable(tableName.getDb(), tableName.getTbl());
     Preconditions.checkState(tbl instanceof View);
-
-    if (!catalog_.tryLockTable(tbl)) {
-      throw new InternalException(String.format("Error altering view %s due to lock " +
-          "contention", tbl.getFullName()));
-    }
+    tryLock(tbl);
     try {
       long newCatalogVersion = catalog_.incrementAndGetCatalogVersion();
       catalog_.getLock().writeLock().unlock();
@@ -3479,12 +3473,18 @@ public class CatalogOpExecutor {
   private void alterCommentOn(TCommentOnParams params, TDdlExecResponse response)
       throws ImpalaRuntimeException, CatalogException, InternalException {
     if (params.getDb() != null) {
-      Preconditions.checkArgument(!params.isSetTable_name());
+      Preconditions.checkArgument(!params.isSetTable_name() &&
+          !params.isSetColumn_name());
       alterCommentOnDb(params.getDb(), params.getComment(), response);
     } else if (params.getTable_name() != null) {
-      Preconditions.checkArgument(!params.isSetDb());
+      Preconditions.checkArgument(!params.isSetDb() && !params.isSetColumn_name());
       alterCommentOnTableOrView(TableName.fromThrift(params.getTable_name()),
           params.getComment(), response);
+    } else if (params.getColumn_name() != null) {
+      Preconditions.checkArgument(!params.isSetDb() && !params.isSetTable_name());
+      TColumnName columnName = params.getColumn_name();
+      alterCommentOnColumn(TableName.fromThrift(columnName.getTable_name()),
+          columnName.getColumn_name(), params.getComment(), response);
     } else {
       throw new UnsupportedOperationException("Unsupported COMMENT ON operation");
     }
@@ -3571,14 +3571,12 @@ public class CatalogOpExecutor {
       TDdlExecResponse response) throws CatalogException, InternalException,
       ImpalaRuntimeException {
     Table tbl = getExistingTable(tableName.getDb(), tableName.getTbl());
-    if (!catalog_.tryLockTable(tbl)) {
-      throw new InternalException(String.format("Error altering table/view %s due to " +
-          "lock contention.", tbl.getFullName()));
-    }
+    tryLock(tbl);
     try {
       long newCatalogVersion = catalog_.incrementAndGetCatalogVersion();
       catalog_.getLock().writeLock().unlock();
-      org.apache.hadoop.hive.metastore.api.Table msTbl = tbl.getMetaStoreTable().deepCopy();
+      org.apache.hadoop.hive.metastore.api.Table msTbl =
+          tbl.getMetaStoreTable().deepCopy();
       boolean isView = msTbl.getTableType().equalsIgnoreCase(
           TableType.VIRTUAL_VIEW.toString());
       if (comment == null) {
@@ -3594,4 +3592,57 @@ public class CatalogOpExecutor {
       tbl.getLock().unlock();
     }
   }
+
+  private void alterCommentOnColumn(TableName tableName, String columnName,
+      String comment, TDdlExecResponse response) throws CatalogException,
+      InternalException, ImpalaRuntimeException {
+    Table tbl = getExistingTable(tableName.getDb(), tableName.getTbl());
+    tryLock(tbl);
+    try {
+      long newCatalogVersion = catalog_.incrementAndGetCatalogVersion();
+      catalog_.getLock().writeLock().unlock();
+      org.apache.hadoop.hive.metastore.api.Table msTbl =
+          tbl.getMetaStoreTable().deepCopy();
+      if (!updateColumnComment(msTbl.getSd().getColsIterator(), columnName, comment)) {
+        if (!updateColumnComment(msTbl.getPartitionKeysIterator(), columnName, comment)) {
+          throw new ColumnNotFoundException(String.format(
+              "Column name %s not found in table %s.", columnName, tbl.getFullName()));
+        }
+      }
+      applyAlterTable(msTbl, true);
+      loadTableMetadata(tbl, newCatalogVersion, false, true, null);
+      addTableToCatalogUpdate(tbl, response.result);
+      addSummary(response, "Column has been altered.");
+    } finally {
+      tbl.getLock().unlock();
+    }
+  }
+
+  /**
+   * Find the matching column name in the iterator and update its comment. Return
+   * true if found; false otherwise.
+   */
+  private static boolean updateColumnComment(Iterator<FieldSchema> iterator,
+      String columnName, String comment) {
+    while (iterator.hasNext()) {
+      FieldSchema fs = iterator.next();
+      if (fs.getName().equalsIgnoreCase(columnName)) {
+        fs.setComment(comment);
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Try to lock a table in the catalog. Throw an InternalException if the catalog is
+   * unable to lock the given table.
+   */
+  private void tryLock(Table tbl) throws InternalException {
+    String type = tbl instanceof View ? "view" : "table";
+    if (!catalog_.tryLockTable(tbl)) {
+      throw new InternalException(String.format("Error altering %s %s due to " +
+          "lock contention.", type, tbl.getFullName()));
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/4c5029af/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
----------------------------------------------------------------------
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 886608a..c565b45 100644
--- a/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
@@ -4014,7 +4014,26 @@ public class AnalyzeDDLTest extends FrontendTestBase {
         "Could not resolve table reference: 'default.doesntexist'");
     AnalysisError("comment on view functional.alltypes is 'comment'",
         "COMMENT ON VIEW not allowed on a table: functional.alltypes");
-    AnalysisError(String.format("comment on table functional.alltypes_view is '%s'",
+    AnalysisError(String.format("comment on view functional.alltypes_view is '%s'",
+        buildLongComment()), "Comment exceeds maximum length of 256 characters. " +
+        "The given comment has 261 characters.");
+  }
+
+  @Test
+  public void TestCommentOnColumn() {
+    for (Pair<String, AnalysisContext> pair : new Pair[]{
+        new Pair<>("functional.alltypes.id", createAnalysisCtx()),
+        new Pair<>("alltypes.id", createAnalysisCtx("functional")),
+        new Pair<>("functional.alltypes_view.id", createAnalysisCtx()),
+        new Pair<>("alltypes_view.id", createAnalysisCtx("functional"))}) {
+      AnalyzesOk(String.format("comment on column %s is 'comment'", pair.first),
+          pair.second);
+      AnalyzesOk(String.format("comment on column %s is ''", pair.first), pair.second);
+      AnalyzesOk(String.format("comment on column %s is null", pair.first), pair.second);
+    }
+    AnalysisError("comment on column functional.alltypes.doesntexist is 'comment'",
+        "Column 'doesntexist' does not exist in table: functional.alltypes");
+    AnalysisError(String.format("comment on column functional.alltypes.id is '%s'",
         buildLongComment()), "Comment exceeds maximum length of 256 characters. " +
         "The given comment has 261 characters.");
   }

http://git-wip-us.apache.org/repos/asf/impala/blob/4c5029af/fe/src/test/java/org/apache/impala/analysis/AuthorizationStmtTest.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/analysis/AuthorizationStmtTest.java b/fe/src/test/java/org/apache/impala/analysis/AuthorizationStmtTest.java
index 0a32fc9..4f88463 100644
--- a/fe/src/test/java/org/apache/impala/analysis/AuthorizationStmtTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/AuthorizationStmtTest.java
@@ -2000,6 +2000,38 @@ public class AuthorizationStmtTest extends FrontendTestBase {
             TPrivilegeLevel.ALL, TPrivilegeLevel.ALTER)))
         .error(alterError("functional.alltypes_view"), onTable("functional",
             "alltypes_view", allExcept(TPrivilegeLevel.ALL, TPrivilegeLevel.ALTER)));
+
+    // Comment on table column.
+    authorize("comment on column functional.alltypes.id is 'comment'")
+        .ok(onServer(TPrivilegeLevel.ALL))
+        .ok(onServer(TPrivilegeLevel.ALTER))
+        .ok(onDatabase("functional", TPrivilegeLevel.ALL))
+        .ok(onDatabase("functional", TPrivilegeLevel.ALTER))
+        .ok(onTable("functional", "alltypes", TPrivilegeLevel.ALL))
+        .ok(onTable("functional", "alltypes", TPrivilegeLevel.ALTER))
+        .error(alterError("functional.alltypes"))
+        .error(alterError("functional.alltypes"), onServer(allExcept(
+            TPrivilegeLevel.ALL, TPrivilegeLevel.ALTER)))
+        .error(alterError("functional.alltypes"), onDatabase("functional", allExcept(
+            TPrivilegeLevel.ALL, TPrivilegeLevel.ALTER)))
+        .error(alterError("functional.alltypes"), onTable("functional", "alltypes",
+            allExcept(TPrivilegeLevel.ALL, TPrivilegeLevel.ALTER)));
+
+    // Comment on view column.
+    authorize("comment on column functional.alltypes_view.id is 'comment'")
+        .ok(onServer(TPrivilegeLevel.ALL))
+        .ok(onServer(TPrivilegeLevel.ALTER))
+        .ok(onDatabase("functional", TPrivilegeLevel.ALL))
+        .ok(onDatabase("functional", TPrivilegeLevel.ALTER))
+        .ok(onTable("functional", "alltypes_view", TPrivilegeLevel.ALL))
+        .ok(onTable("functional", "alltypes_view", TPrivilegeLevel.ALTER))
+        .error(alterError("functional.alltypes_view"))
+        .error(alterError("functional.alltypes_view"), onServer(allExcept(
+            TPrivilegeLevel.ALL, TPrivilegeLevel.ALTER)))
+        .error(alterError("functional.alltypes_view"), onDatabase("functional", allExcept(
+            TPrivilegeLevel.ALL, TPrivilegeLevel.ALTER)))
+        .error(alterError("functional.alltypes_view"), onTable("functional",
+            "alltypes_view", allExcept(TPrivilegeLevel.ALL, TPrivilegeLevel.ALTER)));
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/impala/blob/4c5029af/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
----------------------------------------------------------------------
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 1f19823..6059b8b 100644
--- a/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
@@ -3776,6 +3776,15 @@ public class ParserTest extends FrontendTestBase {
 
     ParserError("COMMENT ON VIEW IS 'comment'");
     ParserError("COMMENT ON VIEW tbl IS");
+
+    for (String col : new String[]{"db.tbl.col", "tbl.col"}) {
+      ParsesOk(String.format("COMMENT ON COLUMN %s IS 'comment'", col));
+      ParsesOk(String.format("COMMENT ON COLUMN %s IS ''", col));
+      ParsesOk(String.format("COMMENT ON COLUMN %s IS NULL", col));
+    }
+    ParserError("COMMENT on col IS 'comment'");
+    ParserError("COMMENT on db.tbl.col IS");
+    ParserError("COMMENT on tbl.col IS");
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/impala/blob/4c5029af/tests/metadata/test_ddl.py
----------------------------------------------------------------------
diff --git a/tests/metadata/test_ddl.py b/tests/metadata/test_ddl.py
index 39f7f3c..7d2b714 100644
--- a/tests/metadata/test_ddl.py
+++ b/tests/metadata/test_ddl.py
@@ -202,14 +202,14 @@ class TestDdlStatements(TestDdlBase):
     comment = self._get_db_comment(unique_database)
     assert 'comment' == comment
 
-    self.client.execute("comment on database {0} is '\\'comment\\''".format(unique_database))
-    comment = self._get_db_comment(unique_database)
-    assert "\\'comment\\'" == comment
-
     self.client.execute("comment on database {0} is ''".format(unique_database))
     comment = self._get_db_comment(unique_database)
     assert '' == comment
 
+    self.client.execute("comment on database {0} is '\\'comment\\''".format(unique_database))
+    comment = self._get_db_comment(unique_database)
+    assert "\\'comment\\'" == comment
+
     self.client.execute("comment on database {0} is null".format(unique_database))
     comment = self._get_db_comment(unique_database)
     assert '' == comment
@@ -274,14 +274,14 @@ class TestDdlStatements(TestDdlBase):
     comment = self._get_table_or_view_comment(table)
     assert "comment" == comment
 
-    self.client.execute("comment on table {0} is '\\'comment\\''".format(table))
-    comment = self._get_table_or_view_comment(table)
-    assert "\\\\'comment\\\\'" == comment
-
     self.client.execute("comment on table {0} is ''".format(table))
     comment = self._get_table_or_view_comment(table)
     assert "" == comment
 
+    self.client.execute("comment on table {0} is '\\'comment\\''".format(table))
+    comment = self._get_table_or_view_comment(table)
+    assert "\\\\'comment\\\\'" == comment
+
     self.client.execute("comment on table {0} is null".format(table))
     comment = self._get_table_or_view_comment(table)
     assert comment is None
@@ -297,18 +297,69 @@ class TestDdlStatements(TestDdlBase):
     comment = self._get_table_or_view_comment(view)
     assert "comment" == comment
 
-    self.client.execute("comment on view {0} is '\\'comment\\''".format(view))
-    comment = self._get_table_or_view_comment(view)
-    assert "\\\\'comment\\\\'" == comment
-
     self.client.execute("comment on view {0} is ''".format(view))
     comment = self._get_table_or_view_comment(view)
     assert "" == comment
 
+    self.client.execute("comment on view {0} is '\\'comment\\''".format(view))
+    comment = self._get_table_or_view_comment(view)
+    assert "\\\\'comment\\\\'" == comment
+
     self.client.execute("comment on view {0} is null".format(view))
     comment = self._get_table_or_view_comment(view)
     assert comment is None
 
+  def test_comment_on_column(self, vector, unique_database):
+    table = "{0}.comment_table".format(unique_database)
+    self.client.execute("create table {0} (i int) partitioned by (j int)".format(table))
+
+    comment = self._get_column_comment(table, 'i')
+    assert '' == comment
+
+    # Updating comment on a regular column.
+    self.client.execute("comment on column {0}.i is 'comment 1'".format(table))
+    comment = self._get_column_comment(table, 'i')
+    assert "comment 1" == comment
+
+    # Updating comment on a partition column.
+    self.client.execute("comment on column {0}.j is 'comment 2'".format(table))
+    comment = self._get_column_comment(table, 'j')
+    assert "comment 2" == comment
+
+    self.client.execute("comment on column {0}.i is ''".format(table))
+    comment = self._get_column_comment(table, 'i')
+    assert "" == comment
+
+    self.client.execute("comment on column {0}.i is '\\'comment\\''".format(table))
+    comment = self._get_column_comment(table, 'i')
+    assert "\\'comment\\'" == comment
+
+    self.client.execute("comment on column {0}.i is null".format(table))
+    comment = self._get_column_comment(table, 'i')
+    assert "" == comment
+
+    view = "{0}.comment_view".format(unique_database)
+    self.client.execute("create view {0}(i) as select 1".format(view))
+
+    comment = self._get_column_comment(view, 'i')
+    assert "" == comment
+
+    self.client.execute("comment on column {0}.i is 'comment'".format(view))
+    comment = self._get_column_comment(view, 'i')
+    assert "comment" == comment
+
+    self.client.execute("comment on column {0}.i is ''".format(view))
+    comment = self._get_column_comment(view, 'i')
+    assert "" == comment
+
+    self.client.execute("comment on column {0}.i is '\\'comment\\''".format(view))
+    comment = self._get_column_comment(view, 'i')
+    assert "\\'comment\\'" == comment
+
+    self.client.execute("comment on column {0}.i is null".format(view))
+    comment = self._get_column_comment(view, 'i')
+    assert "" == comment
+
   @UniqueDatabase.parametrize(sync_ddl=True)
   def test_sync_ddl_drop(self, vector, unique_database):
     """Verifies the catalog gets updated properly when dropping objects with sync_ddl

http://git-wip-us.apache.org/repos/asf/impala/blob/4c5029af/tests/metadata/test_ddl_base.py
----------------------------------------------------------------------
diff --git a/tests/metadata/test_ddl_base.py b/tests/metadata/test_ddl_base.py
index c1388fc..bc74e6e 100644
--- a/tests/metadata/test_ddl_base.py
+++ b/tests/metadata/test_ddl_base.py
@@ -100,4 +100,13 @@ class TestDdlBase(ImpalaTestSuite):
 
   def _get_table_or_view_comment(self, table_name):
     props = self._get_tbl_properties(table_name)
-    return props["comment"] if "comment" in props else None
\ No newline at end of file
+    return props["comment"] if "comment" in props else None
+
+  def _get_column_comment(self, table_or_view_name, col_name):
+    result = self.client.execute("describe {0}".format(table_or_view_name))
+    comments = dict()
+    for row in result.data:
+      cols = row.split('\t')
+      if len(cols) == 3:
+        comments[cols[0].rstrip()] = cols[2].rstrip()
+    return comments.get(col_name)


[2/2] impala git commit: IMPALA-7240: Fix missing QueryMaintenance call in AddBatchStreaming

Posted by tm...@apache.org.
IMPALA-7240: Fix missing QueryMaintenance call in AddBatchStreaming

A recent change, IMPALA-110 (part 2), refactored the aggregation code.
During this refactor, a call to QueryMaintenance() was inadvertantely
dropped in the AddBatchStreaming() path, causing
test_spilling_regression_exhaustive to fail by hitting the memory
limit.

Testing:
- Ran test_spilling_regression_exhaustive

Change-Id: I150a46d00acad139d186a150d9706ef47a10ac36
Reviewed-on: http://gerrit.cloudera.org:8080/10863
Reviewed-by: Michael Ho <kw...@cloudera.com>
Reviewed-by: Tim Armstrong <ta...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/master
Commit: fd0ba0fd2c7631d48b4cce56e60f0b9f902cc446
Parents: 4c5029a
Author: Thomas Tauber-Marshall <tm...@cloudera.com>
Authored: Wed Jul 4 00:00:17 2018 +0000
Committer: Impala Public Jenkins <im...@cloudera.com>
Committed: Thu Jul 5 20:31:02 2018 +0000

----------------------------------------------------------------------
 be/src/exec/grouping-aggregator.cc | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/fd0ba0fd/be/src/exec/grouping-aggregator.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/grouping-aggregator.cc b/be/src/exec/grouping-aggregator.cc
index 60001a6..0eb4a3f 100644
--- a/be/src/exec/grouping-aggregator.cc
+++ b/be/src/exec/grouping-aggregator.cc
@@ -420,6 +420,7 @@ Status GroupingAggregator::AddBatch(RuntimeState* state, RowBatch* batch) {
 Status GroupingAggregator::AddBatchStreaming(
     RuntimeState* state, RowBatch* out_batch, RowBatch* child_batch) {
   SCOPED_TIMER(streaming_timer_);
+  RETURN_IF_ERROR(QueryMaintenance(state));
   num_input_rows_ += child_batch->num_rows();
 
   int remaining_capacity[PARTITION_FANOUT];