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 2018/03/14 21:31:42 UTC

[4/5] impala git commit: IMPALA-6479: Update DESCRIBE to respect column privileges

IMPALA-6479: Update DESCRIBE to respect column privileges

Modified the Frontend to filter columns from the DESCRIBE
statement.  Additionally, if a user has select on at least
one column, they can run DESCRIBE and see most metadata.
If they do not have full table access, they will not see
location or view query metadata.

Testing:
Added tests to validate users that have one or more column
access can run describe and that the output is filtered
accordingly.

Change-Id: Ic96ae184fccdc88ba970b5adcd501da1966accb9
Reviewed-on: http://gerrit.cloudera.org:8080/9276
Reviewed-by: Alex Behm <al...@cloudera.com>
Tested-by: Impala Public Jenkins


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

Branch: refs/heads/2.x
Commit: 9282fa3bac5b2200198daef6455f51f6ad2454c0
Parents: 31de5d8
Author: Adam Holley <gi...@holleyism.com>
Authored: Tue Mar 6 09:50:21 2018 -0600
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Wed Mar 14 04:26:59 2018 +0000

----------------------------------------------------------------------
 be/src/service/client-request-state.cc          |   4 +-
 be/src/service/frontend.cc                      |   9 +-
 be/src/service/frontend.h                       |   5 +-
 common/thrift/Frontend.thrift                   |   3 +
 .../impala/analysis/DescribeTableStmt.java      |  28 +--
 .../org/apache/impala/analysis/InsertStmt.java  |   2 +-
 .../java/org/apache/impala/catalog/Column.java  |  12 ++
 .../java/org/apache/impala/catalog/Table.java   |  26 +--
 .../impala/service/DescribeResultFactory.java   |  33 +--
 .../org/apache/impala/service/Frontend.java     |  58 +++++-
 .../org/apache/impala/service/JniFrontend.java  |   3 +-
 .../apache/impala/analysis/AuditingTest.java    |   4 +-
 .../impala/analysis/AuthorizationTest.java      | 204 ++++++++++++++++++-
 13 files changed, 324 insertions(+), 67 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/9282fa3b/be/src/service/client-request-state.cc
----------------------------------------------------------------------
diff --git a/be/src/service/client-request-state.cc b/be/src/service/client-request-state.cc
index 12bf3b7..655fdb3 100644
--- a/be/src/service/client-request-state.cc
+++ b/be/src/service/client-request-state.cc
@@ -361,8 +361,8 @@ Status ClientRequestState::ExecLocalCatalogOp(
     }
     case TCatalogOpType::DESCRIBE_TABLE: {
       TDescribeResult response;
-      RETURN_IF_ERROR(frontend_->DescribeTable(catalog_op.describe_table_params,
-          &response));
+      const TDescribeTableParams& params = catalog_op.describe_table_params;
+      RETURN_IF_ERROR(frontend_->DescribeTable(params, query_ctx_.session, &response));
       // Set the result set
       request_result_set_.reset(new vector<TResultRow>(response.results));
       return Status::OK();

http://git-wip-us.apache.org/repos/asf/impala/blob/9282fa3b/be/src/service/frontend.cc
----------------------------------------------------------------------
diff --git a/be/src/service/frontend.cc b/be/src/service/frontend.cc
index db54158..ecfae0a 100644
--- a/be/src/service/frontend.cc
+++ b/be/src/service/frontend.cc
@@ -124,8 +124,13 @@ Status Frontend::DescribeDb(const TDescribeDbParams& params,
 }
 
 Status Frontend::DescribeTable(const TDescribeTableParams& params,
-    TDescribeResult* response) {
-  return JniUtil::CallJniMethod(fe_, describe_table_id_, params, response);
+    const TSessionState& session, TDescribeResult* response) {
+  TDescribeTableParams tparams;
+  tparams.__set_output_style(params.output_style);
+  if (params.__isset.table_name) tparams.__set_table_name(params.table_name);
+  if (params.__isset.result_struct) tparams.__set_result_struct(params.result_struct);
+  tparams.__set_session(session);
+  return JniUtil::CallJniMethod(fe_, describe_table_id_, tparams, response);
 }
 
 Status Frontend::ShowCreateTable(const TTableName& table_name, string* response) {

http://git-wip-us.apache.org/repos/asf/impala/blob/9282fa3b/be/src/service/frontend.h
----------------------------------------------------------------------
diff --git a/be/src/service/frontend.h b/be/src/service/frontend.h
index a152b7f..9198b83 100644
--- a/be/src/service/frontend.h
+++ b/be/src/service/frontend.h
@@ -125,8 +125,9 @@ class Frontend {
   /// field is set to MINIMAL, only the column definitions are returned. If set to
   /// FORMATTED|EXTENDED, extended metadata is returned (in addition to the column defs).
   /// This includes info about the table properties, SerDe properties, StorageDescriptor
-  /// properties, and more.
-  Status DescribeTable(const TDescribeTableParams& params, TDescribeResult* response);
+  /// properties, and more.  The current user session is needed for privileges checks.
+  Status DescribeTable(const TDescribeTableParams& params, const TSessionState& session,
+      TDescribeResult* response);
 
   /// Returns (in the output parameter) a string containing the CREATE TABLE command that
   /// creates the table specified in the params.

http://git-wip-us.apache.org/repos/asf/impala/blob/9282fa3b/common/thrift/Frontend.thrift
----------------------------------------------------------------------
diff --git a/common/thrift/Frontend.thrift b/common/thrift/Frontend.thrift
index b81c1a1..837aa88 100644
--- a/common/thrift/Frontend.thrift
+++ b/common/thrift/Frontend.thrift
@@ -166,6 +166,9 @@ struct TDescribeTableParams {
 
   // Set when describing a path to a nested collection.
   3: optional Types.TColumnType result_struct
+
+  // Session state for the user who initiated this request.
+  4: optional ImpalaInternalService.TSessionState session
 }
 
 // Results of a call to describeDb() and describeTable()

http://git-wip-us.apache.org/repos/asf/impala/blob/9282fa3b/fe/src/main/java/org/apache/impala/analysis/DescribeTableStmt.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/DescribeTableStmt.java b/fe/src/main/java/org/apache/impala/analysis/DescribeTableStmt.java
index 90f8b07..9de5054 100644
--- a/fe/src/main/java/org/apache/impala/analysis/DescribeTableStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/DescribeTableStmt.java
@@ -23,6 +23,7 @@ import java.util.List;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.impala.analysis.Path.PathType;
 import org.apache.impala.authorization.Privilege;
+import org.apache.impala.authorization.PrivilegeRequest;
 import org.apache.impala.authorization.PrivilegeRequestBuilder;
 import org.apache.impala.catalog.StructType;
 import org.apache.impala.catalog.Table;
@@ -86,21 +87,6 @@ public class DescribeTableStmt extends StatementBase {
   public Table getTable() { return table_; }
   public TDescribeOutputStyle getOutputStyle() { return outputStyle_; }
 
-  /**
-   * Get the privilege requirement, which depends on the output style.
-   */
-  private Privilege getPrivilegeRequirement() {
-    switch (outputStyle_) {
-      case MINIMAL: return Privilege.ANY;
-      case FORMATTED:
-      case EXTENDED:
-        return Privilege.VIEW_METADATA;
-      default:
-        Preconditions.checkArgument(false);
-        return null;
-    }
-  }
-
   @Override
   public void collectTableRefs(List<TableRef> tblRefs) {
     tblRefs.add(new TableRef(rawPath_, null));
@@ -116,12 +102,10 @@ public class DescribeTableStmt extends StatementBase {
       // table/database if the user is not authorized.
       if (rawPath_.size() > 1) {
         analyzer.registerPrivReq(new PrivilegeRequestBuilder()
-            .onTable(rawPath_.get(0), rawPath_.get(1))
-            .allOf(getPrivilegeRequirement()).toRequest());
+            .onTable(rawPath_.get(0), rawPath_.get(1)).any().toRequest());
       }
       analyzer.registerPrivReq(new PrivilegeRequestBuilder()
-          .onTable(analyzer.getDefaultDb(), rawPath_.get(0))
-          .allOf(getPrivilegeRequirement()).toRequest());
+          .onTable(analyzer.getDefaultDb(), rawPath_.get(0)).any().toRequest());
       throw ae;
     } catch (TableLoadingException tle) {
       throw new AnalysisException(tle.getMessage(), tle);
@@ -129,11 +113,15 @@ public class DescribeTableStmt extends StatementBase {
 
     table_ = path_.getRootTable();
     // Register authorization and audit events.
-    analyzer.getTable(table_.getTableName(), getPrivilegeRequirement());
+    analyzer.getTable(table_.getTableName(), Privilege.ANY);
 
     // Describing a table.
     if (path_.destTable() != null) return;
 
+    analyzer.registerPrivReq(new PrivilegeRequestBuilder()
+        .onColumn(path_.getRootTable().getDb().getName(), path_.getRootTable().getName(),
+        path_.getRawPath().get(0)).any().toRequest());
+
     if (path_.destType().isComplexType()) {
       if (outputStyle_ == TDescribeOutputStyle.FORMATTED ||
           outputStyle_ == TDescribeOutputStyle.EXTENDED) {

http://git-wip-us.apache.org/repos/asf/impala/blob/9282fa3b/fe/src/main/java/org/apache/impala/analysis/InsertStmt.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/InsertStmt.java b/fe/src/main/java/org/apache/impala/analysis/InsertStmt.java
index 4f8d44b..20116c2 100644
--- a/fe/src/main/java/org/apache/impala/analysis/InsertStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/InsertStmt.java
@@ -724,7 +724,7 @@ public class InsertStmt extends StatementBase {
 
     // Finally, 'undo' the permutation so that the selectListExprs are in Hive column
     // order, and add NULL expressions to all missing columns, unless this is an UPSERT.
-    ArrayList<Column> columns = table_.getColumnsInHiveOrder();
+    List<Column> columns = table_.getColumnsInHiveOrder();
     for (int col = 0; col < columns.size(); ++col) {
       Column tblColumn = columns.get(col);
       boolean matchFound = false;

http://git-wip-us.apache.org/repos/asf/impala/blob/9282fa3b/fe/src/main/java/org/apache/impala/catalog/Column.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/Column.java b/fe/src/main/java/org/apache/impala/catalog/Column.java
index 0f4086c..bec5852 100644
--- a/fe/src/main/java/org/apache/impala/catalog/Column.java
+++ b/fe/src/main/java/org/apache/impala/catalog/Column.java
@@ -17,6 +17,7 @@
 
 package org.apache.impala.catalog;
 
+import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData;
@@ -136,4 +137,15 @@ public class Column {
     for (Column col: columns) colNames.add(col.getName());
     return colNames;
   }
+  /**
+   * Returns a struct type from the table columns passed in as a parameter.
+   */
+  public static StructType columnsToStruct(List<Column> columns) {
+    ArrayList<StructField> fields = Lists.newArrayListWithCapacity(columns.size());
+    for (Column col: columns) {
+      fields.add(new StructField(col.getName(), col.getType(), col.getComment()));
+    }
+    return new StructType(fields);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/9282fa3b/fe/src/main/java/org/apache/impala/catalog/Table.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/Table.java b/fe/src/main/java/org/apache/impala/catalog/Table.java
index aca9409..9255f0a 100644
--- a/fe/src/main/java/org/apache/impala/catalog/Table.java
+++ b/fe/src/main/java/org/apache/impala/catalog/Table.java
@@ -18,6 +18,7 @@
 package org.apache.impala.catalog;
 
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -472,39 +473,30 @@ public abstract class Table extends CatalogObjectImpl {
   public String getStorageHandlerClassName() { return null; }
 
   /**
-   * Returns the list of all columns, but with partition columns at the end of
+   * Returns an unmodifiable list of all columns, but with partition columns at the end of
    * the list rather than the beginning. This is equivalent to the order in
    * which Hive enumerates columns.
    */
-  public ArrayList<Column> getColumnsInHiveOrder() {
+  public List<Column> getColumnsInHiveOrder() {
     ArrayList<Column> columns = Lists.newArrayList(getNonClusteringColumns());
     columns.addAll(getClusteringColumns());
-    return columns;
+    return Collections.unmodifiableList(columns);
   }
 
-  /**
-   * Returns a struct type with the columns in the same order as getColumnsInHiveOrder().
-   */
-  public StructType getHiveColumnsAsStruct() {
-    ArrayList<StructField> fields = Lists.newArrayListWithCapacity(colsByPos_.size());
-    for (Column col: getColumnsInHiveOrder()) {
-      fields.add(new StructField(col.getName(), col.getType(), col.getComment()));
-    }
-    return new StructType(fields);
-  }
 
   /**
-   * Returns the list of all partition columns.
+   * Returns an unmodifiable list of all partition columns.
    */
   public List<Column> getClusteringColumns() {
-    return colsByPos_.subList(0, numClusteringCols_);
+    return Collections.unmodifiableList(colsByPos_.subList(0, numClusteringCols_));
   }
 
   /**
-   * Returns the list of all columns excluding any partition columns.
+   * Returns an unmodifiable list of all columns excluding any partition columns.
    */
   public List<Column> getNonClusteringColumns() {
-    return colsByPos_.subList(numClusteringCols_, colsByPos_.size());
+    return Collections.unmodifiableList(colsByPos_.subList(numClusteringCols_,
+        colsByPos_.size()));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/impala/blob/9282fa3b/fe/src/main/java/org/apache/impala/service/DescribeResultFactory.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/service/DescribeResultFactory.java b/fe/src/main/java/org/apache/impala/service/DescribeResultFactory.java
index 5e90b06..9a5adf4 100644
--- a/fe/src/main/java/org/apache/impala/service/DescribeResultFactory.java
+++ b/fe/src/main/java/org/apache/impala/service/DescribeResultFactory.java
@@ -17,6 +17,7 @@
 
 package org.apache.impala.service;
 
+import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
@@ -28,7 +29,6 @@ import org.apache.hadoop.hive.ql.metadata.formatting.MetaDataFormatUtils;
 import org.apache.impala.catalog.Column;
 import org.apache.impala.catalog.Db;
 import org.apache.impala.catalog.KuduColumn;
-import org.apache.impala.catalog.KuduTable;
 import org.apache.impala.catalog.StructField;
 import org.apache.impala.catalog.StructType;
 import org.apache.impala.catalog.Table;
@@ -181,9 +181,11 @@ public class DescribeResultFactory {
    * Builds a TDescribeResult that contains the result of a DESCRIBE FORMATTED|EXTENDED
    * <table> command. For the formatted describe output the goal is to be exactly the
    * same as what Hive (via HiveServer2) outputs, for compatibility reasons. To do this,
-   * Hive's MetadataFormatUtils class is used to build the results.
+   * Hive's MetadataFormatUtils class is used to build the results.  filteredColumns is a
+   * list of columns the user is authorized to view.
    */
-  public static TDescribeResult buildDescribeFormattedResult(Table table) {
+  public static TDescribeResult buildDescribeFormattedResult(Table table,
+      List<Column> filteredColumns) {
     TDescribeResult result = new TDescribeResult();
     result.results = Lists.newArrayList();
 
@@ -192,8 +194,19 @@ public class DescribeResultFactory {
     // For some table formats (e.g. Avro) the column list in the table can differ from the
     // one returned by the Hive metastore. To handle this we use the column list from the
     // table which has already reconciled those differences.
-    msTable.getSd().setCols(Column.toFieldSchemas(table.getNonClusteringColumns()));
-    msTable.setPartitionKeys(Column.toFieldSchemas(table.getClusteringColumns()));
+    // Need to create a new list since if the columns are filtered, this will
+    // affect the original list.
+    List<Column> nonClustered = new ArrayList<Column>();
+    List<Column> clustered = new ArrayList<Column>();
+    for (Column col: filteredColumns) {
+      if (table.isClusteringColumn(col)) {
+        clustered.add(col);
+      } else {
+        nonClustered.add(col);
+      }
+    }
+    msTable.getSd().setCols(Column.toFieldSchemas(nonClustered));
+    msTable.setPartitionKeys(Column.toFieldSchemas(clustered));
 
     // To avoid initializing any of the SerDe classes in the metastore table Thrift
     // struct, create the ql.metadata.Table object by calling the empty c'tor and
@@ -248,16 +261,12 @@ public class DescribeResultFactory {
   }
 
   /**
-   * Builds a TDescribeResult for a table.
+   * Builds a TDescribeResult for a kudu table from a list of columns.
    */
-  public static TDescribeResult buildDescribeMinimalResult(Table table) {
-    if (!(table instanceof KuduTable)) {
-      return buildDescribeMinimalResult(table.getHiveColumnsAsStruct());
-    }
-
+  public static TDescribeResult buildKuduDescribeMinimalResult(List<Column> columns) {
     TDescribeResult descResult = new TDescribeResult();
     descResult.results = Lists.newArrayList();
-    for (Column c: table.getColumnsInHiveOrder()) {
+    for (Column c: columns) {
       Preconditions.checkState(c instanceof KuduColumn);
       KuduColumn kuduColumn = (KuduColumn) c;
       // General describe info.

http://git-wip-us.apache.org/repos/asf/impala/blob/9282fa3b/fe/src/main/java/org/apache/impala/service/Frontend.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/service/Frontend.java b/fe/src/main/java/org/apache/impala/service/Frontend.java
index 41395d6..539fe31 100644
--- a/fe/src/main/java/org/apache/impala/service/Frontend.java
+++ b/fe/src/main/java/org/apache/impala/service/Frontend.java
@@ -66,6 +66,7 @@ import org.apache.impala.authorization.AuthorizationChecker;
 import org.apache.impala.authorization.AuthorizationConfig;
 import org.apache.impala.authorization.AuthorizeableTable;
 import org.apache.impala.authorization.ImpalaInternalAdminUser;
+import org.apache.impala.authorization.Privilege;
 import org.apache.impala.authorization.PrivilegeRequest;
 import org.apache.impala.authorization.PrivilegeRequestBuilder;
 import org.apache.impala.authorization.User;
@@ -786,14 +787,65 @@ public class Frontend {
    * the table metadata.
    */
   public TDescribeResult describeTable(TTableName tableName,
-      TDescribeOutputStyle outputStyle) throws ImpalaException {
+      TDescribeOutputStyle outputStyle, User user) throws ImpalaException {
     Table table = impaladCatalog_.get().getTable(tableName.db_name, tableName.table_name);
+    List<Column> filteredColumns;
+    if (authzConfig_.isEnabled()) {
+      // First run a table check
+      PrivilegeRequest privilegeRequest = new PrivilegeRequestBuilder()
+          .allOf(Privilege.SELECT).onTable(table.getDb().getName(), table.getName())
+          .toRequest();
+      if (!authzChecker_.get().hasAccess(user, privilegeRequest)) {
+        // Filter out columns that the user is not authorized to see.
+        filteredColumns = new ArrayList<Column>();
+        for (Column col: table.getColumnsInHiveOrder()) {
+          String colName = col.getName();
+          privilegeRequest = new PrivilegeRequestBuilder()
+              .allOf(Privilege.SELECT)
+              .onColumn(table.getDb().getName(), table.getName(), colName)
+              .toRequest();
+          if (authzChecker_.get().hasAccess(user, privilegeRequest)) {
+            filteredColumns.add(col);
+          }
+        }
+      } else {
+        // User has table-level access
+        filteredColumns = table.getColumnsInHiveOrder();
+      }
+    } else {
+      // Authorization is disabled
+      filteredColumns = table.getColumnsInHiveOrder();
+    }
     if (outputStyle == TDescribeOutputStyle.MINIMAL) {
-      return DescribeResultFactory.buildDescribeMinimalResult(table);
+      if (!(table instanceof KuduTable)) {
+        return DescribeResultFactory.buildDescribeMinimalResult(
+            Column.columnsToStruct(filteredColumns));
+      }
+      return DescribeResultFactory.buildKuduDescribeMinimalResult(filteredColumns);
     } else {
       Preconditions.checkArgument(outputStyle == TDescribeOutputStyle.FORMATTED ||
           outputStyle == TDescribeOutputStyle.EXTENDED);
-      return DescribeResultFactory.buildDescribeFormattedResult(table);
+      TDescribeResult result = DescribeResultFactory.buildDescribeFormattedResult(table,
+          filteredColumns);
+      // Filter out LOCATION text
+      if (authzConfig_.isEnabled()) {
+        PrivilegeRequest privilegeRequest = new PrivilegeRequestBuilder()
+            .allOf(Privilege.VIEW_METADATA)
+            .onTable(table.getDb().getName(),table.getName())
+            .toRequest();
+        // Only filter if the user doesn't have table access.
+        if (!authzChecker_.get().hasAccess(user, privilegeRequest)) {
+          List<TResultRow> results = new ArrayList<TResultRow>();
+          for(TResultRow row: result.getResults()) {
+            String stringVal = row.getColVals().get(0).getString_val();
+            if (!stringVal.contains("Location")) {
+              results.add(row);
+            }
+          }
+          result.setResults(results);
+        }
+      }
+      return result;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/impala/blob/9282fa3b/fe/src/main/java/org/apache/impala/service/JniFrontend.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/service/JniFrontend.java b/fe/src/main/java/org/apache/impala/service/JniFrontend.java
index 3d99a4a..20a028d 100644
--- a/fe/src/main/java/org/apache/impala/service/JniFrontend.java
+++ b/fe/src/main/java/org/apache/impala/service/JniFrontend.java
@@ -431,9 +431,10 @@ public class JniFrontend {
     JniUtil.deserializeThrift(protocolFactory_, params, thriftDescribeTableParams);
 
     Preconditions.checkState(params.isSetTable_name() ^ params.isSetResult_struct());
+    User user = new User(TSessionStateUtil.getEffectiveUser(params.getSession()));
     TDescribeResult result = null;
     if (params.isSetTable_name()) {
-      result = frontend_.describeTable(params.getTable_name(), params.output_style);
+      result = frontend_.describeTable(params.getTable_name(), params.output_style, user);
     } else {
       Preconditions.checkState(params.output_style == TDescribeOutputStyle.MINIMAL);
       StructType structType = (StructType)Type.fromThrift(params.result_struct);

http://git-wip-us.apache.org/repos/asf/impala/blob/9282fa3b/fe/src/test/java/org/apache/impala/analysis/AuditingTest.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/analysis/AuditingTest.java b/fe/src/test/java/org/apache/impala/analysis/AuditingTest.java
index 610bf56..cc91188 100644
--- a/fe/src/test/java/org/apache/impala/analysis/AuditingTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/AuditingTest.java
@@ -313,7 +313,7 @@ public class AuditingTest extends FrontendTestBase {
 
     accessEvents = AnalyzeAccessEvents("describe formatted functional.alltypesagg");
     Assert.assertEquals(accessEvents, Sets.newHashSet(new TAccessEvent(
-        "functional.alltypesagg", TCatalogObjectType.TABLE, "VIEW_METADATA")));
+        "functional.alltypesagg", TCatalogObjectType.TABLE, "ANY")));
 
     accessEvents = AnalyzeAccessEvents("describe functional.complex_view");
     Assert.assertEquals(accessEvents, Sets.newHashSet(new TAccessEvent(
@@ -453,7 +453,7 @@ public class AuditingTest extends FrontendTestBase {
     // Describe formatted
     accessEvents = AnalyzeAccessEvents("describe formatted functional_kudu.testtbl");
     Assert.assertEquals(accessEvents, Sets.newHashSet(new TAccessEvent(
-        "functional_kudu.testtbl", TCatalogObjectType.TABLE, "VIEW_METADATA")));
+        "functional_kudu.testtbl", TCatalogObjectType.TABLE, "ANY")));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/impala/blob/9282fa3b/fe/src/test/java/org/apache/impala/analysis/AuthorizationTest.java
----------------------------------------------------------------------
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 bcea229..a3ff4c1 100644
--- a/fe/src/test/java/org/apache/impala/analysis/AuthorizationTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/AuthorizationTest.java
@@ -48,6 +48,9 @@ import org.apache.impala.common.InternalException;
 import org.apache.impala.common.RuntimeEnv;
 import org.apache.impala.service.Frontend;
 import org.apache.impala.testutil.ImpaladTestCatalog;
+import org.apache.impala.thrift.TColumnValue;
+import org.apache.impala.thrift.TDescribeOutputStyle;
+import org.apache.impala.thrift.TDescribeResult;
 import org.apache.impala.thrift.TFunctionBinaryType;
 import org.apache.impala.thrift.TMetadataOpRequest;
 import org.apache.impala.thrift.TMetadataOpcode;
@@ -55,8 +58,10 @@ import org.apache.impala.thrift.TNetworkAddress;
 import org.apache.impala.thrift.TPrivilege;
 import org.apache.impala.thrift.TPrivilegeLevel;
 import org.apache.impala.thrift.TPrivilegeScope;
+import org.apache.impala.thrift.TResultRow;
 import org.apache.impala.thrift.TResultSet;
 import org.apache.impala.thrift.TSessionState;
+import org.apache.impala.thrift.TTableName;
 import org.apache.impala.util.PatternMatcher;
 import org.apache.impala.util.SentryPolicyService;
 import org.apache.sentry.provider.common.ResourceAuthorizationProvider;
@@ -1459,13 +1464,15 @@ public class AuthorizationTest extends FrontendTestBase {
     AuthzOk("describe functional.alltypessmall");
     // User has column level privileges on described column.
     AuthzOk("describe functional.allcomplextypes.int_struct_col");
-    // User has column level privileges on another column in table.
-    AuthzOk("describe functional.allcomplextypes.complex_struct_col");
+    // User has column level privileges on another column in table but not this one.
+    AuthzError("describe functional.allcomplextypes.complex_struct_col",
+        "User '%s' does not have privileges to access: functional.allcomplextypes");
     // User has table level privileges without column level.
     AuthzOk("describe functional_parquet.allcomplextypes.complex_struct_col");
-    // Insufficient privileges on table.
-    AuthzError("describe formatted functional.alltypestiny",
-        "User '%s' does not have privileges to access: functional.alltypestiny");
+    // Column level privileges will allow describe but with reduced data.
+    AuthzOk("describe formatted functional.alltypestiny");
+    // Column level privileges will allow describe but with reduced data.
+    AuthzOk("describe formatted functional.alltypessmall");
     // Insufficient privileges on table for nested column.
     AuthzError("describe functional.complextypestbl.nested_struct",
         "User '%s' does not have privileges to access: functional.complextypestbl");
@@ -1475,6 +1482,193 @@ public class AuthorizationTest extends FrontendTestBase {
     // Insufficient privileges on db.
     AuthzError("describe functional_rc.alltypes",
         "User '%s' does not have privileges to access: functional_rc.alltypes");
+    // Insufficient privileges on column that is a complex type, trying to access member.
+    AuthzError("describe functional.allcomplextypes.complex_struct_col.f2",
+        "User '%s' does not have privileges to access: functional.allcomplextypes");
+    // Insufficient privileges on column that is not a complex type, trying to access member.
+    AuthzError("describe functional.allcomplextypes.nested_struct_col.f1",
+        "User '%s' does not have privileges to access: functional.allcomplextypes");
+  }
+
+  // Expected output of DESCRIBE for a functional table.
+  private static final List<String> EXPECTED_DESCRIBE_ALLTYPESSMALL =
+      Lists.newArrayList(
+      "id","int","",
+      "int_col","int", "",
+      "year","int", ""
+      );
+
+  // Expected output of DESCRIBE for a functional table.
+  private static final List<String> EXPECTED_DESCRIBE_ALLTYPESAGG =
+      Lists.newArrayList(
+      "id","int","",
+      "bool_col","boolean","",
+      "tinyint_col","tinyint","",
+      "smallint_col","smallint", "",
+      "int_col","int", "",
+      "bigint_col","bigint", "",
+      "float_col","float", "",
+      "double_col","double", "",
+      "date_string_col","string", "",
+      "string_col","string", "",
+      "timestamp_col","timestamp", "",
+      "year","int", "",
+      "month","int", "",
+      "day","int", ""
+      );
+
+  // Expected output of DESCRIBE for a functional table.
+  // "*" is used when the output is variable such as time or user.
+  private static final List<String> EXPECTED_DESCRIBE_EXTENDED_ALLTYPESAGG =
+      Lists.newArrayList(
+      "# col_name","data_type","comment",
+      "","NULL","NULL",
+      "id","int","NULL",
+      "bool_col","boolean","NULL",
+      "tinyint_col","tinyint","NULL",
+      "smallint_col","smallint","NULL",
+      "int_col","int","NULL",
+      "bigint_col","bigint","NULL",
+      "float_col","float","NULL",
+      "double_col","double","NULL",
+      "date_string_col","string","NULL",
+      "string_col","string","NULL",
+      "timestamp_col","timestamp","NULL",
+      "","NULL","NULL",
+      "# Partition Information","NULL","NULL",
+      "# col_name","data_type","comment",
+      "","NULL","NULL",
+      "year","int","NULL",
+      "month","int","NULL",
+      "day","int","NULL",
+      "","NULL","NULL",
+      "# Detailed Table Information","NULL","NULL",
+      "Database:","functional","NULL",
+      "Owner:","*","NULL",
+      "CreateTime:","*","NULL",
+      "LastAccessTime:","UNKNOWN","NULL",
+      "Protect Mode:","None","NULL",
+      "Retention:","0","NULL",
+      "Location:","hdfs://localhost:20500/test-warehouse/alltypesagg","NULL",
+      "Table Type:","EXTERNAL_TABLE","NULL",
+      "Table Parameters:","NULL","NULL",
+      "","DO_NOT_UPDATE_STATS","true",
+      "","EXTERNAL","TRUE",
+      "","STATS_GENERATED_VIA_STATS_TASK","true",
+      "","numRows","11000",
+      "","totalSize","834279",
+      "","transient_lastDdlTime","*",
+      "","NULL","NULL",
+      "# Storage Information","NULL","NULL",
+      "SerDe Library:","org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","NULL",
+      "InputFormat:","org.apache.hadoop.mapred.TextInputFormat","NULL",
+      "OutputFormat:","org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","NULL",
+      "Compressed:","No","NULL",
+      "Num Buckets:","0","NULL",
+      "Bucket Columns:","[]","NULL",
+      "Sort Columns:","[]","NULL",
+      "Storage Desc Params:","NULL","NULL",
+      "","escape.delim","\\\\",
+      "","field.delim",",",
+      "","serialization.format",","
+      );
+
+  // Expected output of DESCRIBE for a functional table.
+  // "*" is used when the output is variable such as time or user.
+  private static final List<String> EXPECTED_DESCRIBE_EXTENDED_ALLTYPESSMALL =
+      Lists.newArrayList(
+      "# col_name","data_type","comment",
+      "","NULL","NULL",
+      "id","int","NULL",
+      "int_col","int","NULL",
+      "","NULL","NULL",
+      "# Partition Information","NULL","NULL",
+      "# col_name","data_type","comment",
+      "","NULL","NULL",
+      "year","int","NULL",
+      "","NULL","NULL",
+      "# Detailed Table Information","NULL","NULL",
+      "Database:","functional","NULL",
+      "Owner:","*","NULL",
+      "CreateTime:","*","NULL",
+      "LastAccessTime:","UNKNOWN","NULL",
+      "Protect Mode:","None","NULL",
+      "Retention:","0","NULL",
+      "Table Type:","EXTERNAL_TABLE","NULL",
+      "Table Parameters:","NULL","NULL",
+      "","DO_NOT_UPDATE_STATS","true",
+      "","EXTERNAL","TRUE",
+      "","STATS_GENERATED_VIA_STATS_TASK","true",
+      "","numRows","100",
+      "","totalSize","6472",
+      "","transient_lastDdlTime","*",
+      "","NULL","NULL",
+      "# Storage Information","NULL","NULL",
+      "SerDe Library:","org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","NULL",
+      "InputFormat:","org.apache.hadoop.mapred.TextInputFormat","NULL",
+      "OutputFormat:","org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","NULL",
+      "Compressed:","No","NULL",
+      "Num Buckets:","0","NULL",
+      "Bucket Columns:","[]","NULL",
+      "Sort Columns:","[]","NULL",
+      "Storage Desc Params:","NULL","NULL",
+      "","escape.delim","\\\\",
+      "","field.delim",",",
+      "","serialization.format",","
+      );
+
+  @Test
+  public void TestDescribeTableResults() throws ImpalaException {
+    // Verify MINIMAL describe contains all columns
+    TDescribeResult result = fe_.describeTable(new TTableName("functional","alltypesagg"),
+        TDescribeOutputStyle.MINIMAL, USER);
+    Assert.assertEquals(EXPECTED_DESCRIBE_ALLTYPESAGG, resultToStringList(result));
+
+    // Verify EXTENDED output contains all columns and data
+    result = fe_.describeTable(new TTableName("functional","alltypesagg"),
+        TDescribeOutputStyle.EXTENDED, USER);
+    verifyOutputWithOptionalData(EXPECTED_DESCRIBE_EXTENDED_ALLTYPESAGG,
+        resultToStringList(result));
+
+    // Verify FORMATTED output contains all columns and data
+    result = fe_.describeTable(new TTableName("functional","alltypesagg"),
+        TDescribeOutputStyle.FORMATTED, USER);
+    verifyOutputWithOptionalData(EXPECTED_DESCRIBE_EXTENDED_ALLTYPESAGG,
+        resultToStringList(result));
+
+    // Verify MINIMAL describe on restricted table shows limited columns.
+    result = fe_.describeTable(new TTableName("functional","alltypessmall"),
+        TDescribeOutputStyle.MINIMAL, USER);
+    Assert.assertEquals(EXPECTED_DESCRIBE_ALLTYPESSMALL,
+        resultToStringList(result));
+
+    // Verify FORMATTED output contains all columns and data
+    result = fe_.describeTable(new TTableName("functional","alltypessmall"),
+        TDescribeOutputStyle.FORMATTED, USER);
+    verifyOutputWithOptionalData(EXPECTED_DESCRIBE_EXTENDED_ALLTYPESSMALL,
+        resultToStringList(result));
+  }
+
+  // Compares two arrays but skips an expected value that contains '*' since we need to
+  // compare output but some values change based on builds, environments, etc.
+  private void verifyOutputWithOptionalData(List<String> expected, List<String> actual) {
+    Assert.assertEquals(expected.size(), actual.size());
+    for (int idx = 0; idx < expected.size(); idx++) {
+      if (!expected.get(idx).equals("*")) {
+        Assert.assertEquals(expected.get(idx), actual.get(idx));
+      }
+    }
+  }
+
+  // Convert TDescribeResult to ArrayList of strings.
+  private static List<String> resultToStringList(TDescribeResult result) {
+    List<String> strarr = new ArrayList<String>();
+    for (TResultRow row: result.getResults()) {
+      for (TColumnValue col: row.getColVals()) {
+        strarr.add(col.getString_val() == null ? "NULL": col.getString_val().trim());
+      }
+    }
+    return strarr;
   }
 
   @Test