You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@drill.apache.org by GitBox <gi...@apache.org> on 2019/01/04 06:54:07 UTC

[GitHub] gparai closed pull request #1559: DRILL-540: Allow querying hive views in Drill

gparai closed pull request #1559: DRILL-540: Allow querying hive views in Drill
URL: https://github.com/apache/drill/pull/1559
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/common/src/main/java/org/apache/drill/common/AutoCloseables.java b/common/src/main/java/org/apache/drill/common/AutoCloseables.java
index 8ca715e6d8f..4cdbded2dbc 100644
--- a/common/src/main/java/org/apache/drill/common/AutoCloseables.java
+++ b/common/src/main/java/org/apache/drill/common/AutoCloseables.java
@@ -19,12 +19,18 @@
 
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.Objects;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Utilities for AutoCloseable classes.
  */
 public class AutoCloseables {
 
+  private static final Logger LOGGER = LoggerFactory.getLogger(AutoCloseables.class);
+
   public interface Closeable extends AutoCloseable {
     @Override
     void close();
@@ -92,4 +98,20 @@ public static void close(Iterable<? extends AutoCloseable> ac) throws Exception
       throw topLevelException;
     }
   }
+
+  /**
+   * Close all without caring about thrown exceptions
+   * @param closeables - array containing auto closeables
+   */
+  public static void closeSilently(AutoCloseable... closeables) {
+    Arrays.stream(closeables).filter(Objects::nonNull)
+        .forEach(target -> {
+          try {
+            target.close();
+          } catch (Exception e) {
+            LOGGER.warn(String.format("Exception was thrown while closing auto closeable: %s", target), e);
+          }
+        });
+  }
+
 }
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/types/HiveToRelDataTypeConverter.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/types/HiveToRelDataTypeConverter.java
new file mode 100644
index 00000000000..3b3abf28edc
--- /dev/null
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/types/HiveToRelDataTypeConverter.java
@@ -0,0 +1,158 @@
+/*
+ * 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.drill.exec.planner.types;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.SqlCollation;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.Util;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.MapTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class is responsible for data type conversions
+ * from {@link org.apache.hadoop.hive.metastore.api.FieldSchema} instances
+ * to {@link  org.apache.calcite.rel.type.RelDataType} instances
+ */
+public class HiveToRelDataTypeConverter {
+
+  private static final Logger logger = LoggerFactory.getLogger(HiveToRelDataTypeConverter.class);
+
+  private static final String UNSUPPORTED_HIVE_DATA_TYPE_ERROR_MSG = "Unsupported Hive data type %s. %n" +
+      "Following Hive data types are supported in Drill INFORMATION_SCHEMA: " +
+      "BOOLEAN, BYTE, SHORT, INT, LONG, FLOAT, DOUBLE, DATE, TIMESTAMP, BINARY, DECIMAL, STRING, " +
+      "VARCHAR, CHAR, LIST, MAP, STRUCT and UNION";
+
+
+  private final RelDataTypeFactory typeFactory;
+
+  public HiveToRelDataTypeConverter(RelDataTypeFactory typeFactory) {
+    this.typeFactory = typeFactory;
+  }
+
+  /**
+   * Performs conversion from Hive field to nullable RelDataType
+   *
+   * @param field - representation of data type in Hive Metastore
+   * @return appropriate nullable RelDataType for using with Calcite
+   * @throws RuntimeException for unsupported data types, check
+   *         {@link HiveToRelDataTypeConverter#UNSUPPORTED_HIVE_DATA_TYPE_ERROR_MSG}
+   *         for details about supported hive types
+   */
+  public RelDataType convertToNullableRelDataType(FieldSchema field) {
+    TypeInfo fieldTypeInfo = TypeInfoUtils.getTypeInfoFromTypeString(field.getType());
+    RelDataType relDataType = convertToRelDataType(fieldTypeInfo);
+    return typeFactory.createTypeWithNullability(relDataType, true);
+  }
+
+  private RelDataType convertToRelDataType(TypeInfo typeInfo) {
+    final Category typeCategory = typeInfo.getCategory();
+    switch (typeCategory) {
+      case PRIMITIVE:
+        return getRelDataType((PrimitiveTypeInfo) typeInfo);
+      case LIST:
+        return getRelDataType((ListTypeInfo) typeInfo);
+      case MAP:
+        return getRelDataType((MapTypeInfo) typeInfo);
+      case STRUCT:
+        return getRelDataType((StructTypeInfo) typeInfo);
+      case UNION:
+        logger.warn("There is no UNION data type in SQL. Converting it to Sql type OTHER to avoid " +
+            "breaking INFORMATION_SCHEMA queries");
+        return typeFactory.createSqlType(SqlTypeName.OTHER);
+    }
+    throw new RuntimeException(String.format(UNSUPPORTED_HIVE_DATA_TYPE_ERROR_MSG, typeCategory));
+  }
+
+  private RelDataType getRelDataType(StructTypeInfo structTypeInfo) {
+    final List<String> fieldNames = structTypeInfo.getAllStructFieldNames();
+    final List<RelDataType> relDataTypes = structTypeInfo.getAllStructFieldTypeInfos().stream()
+        .map(this::convertToRelDataType)
+        .collect(Collectors.toList());
+    return typeFactory.createStructType(relDataTypes, fieldNames);
+  }
+
+  private RelDataType getRelDataType(MapTypeInfo mapTypeInfo) {
+    RelDataType keyType = convertToRelDataType(mapTypeInfo.getMapKeyTypeInfo());
+    RelDataType valueType = convertToRelDataType(mapTypeInfo.getMapValueTypeInfo());
+    return typeFactory.createMapType(keyType, valueType);
+  }
+
+  private RelDataType getRelDataType(ListTypeInfo listTypeInfo) {
+    RelDataType listElemTypeInfo = convertToRelDataType(listTypeInfo.getListElementTypeInfo());
+    return typeFactory.createArrayType(listElemTypeInfo, -1);
+  }
+
+  private RelDataType getRelDataType(PrimitiveTypeInfo primitiveTypeInfo) {
+    final PrimitiveObjectInspector.PrimitiveCategory primitiveCategory = primitiveTypeInfo.getPrimitiveCategory();
+    switch (primitiveCategory) {
+      case STRING:
+      case VARCHAR:
+        return getRelDataType(primitiveTypeInfo, SqlTypeName.VARCHAR);
+      case CHAR:
+        return getRelDataType(primitiveTypeInfo, SqlTypeName.CHAR);
+      case BYTE:
+      case SHORT:
+      case INT:
+        return typeFactory.createSqlType(SqlTypeName.INTEGER);
+      case DECIMAL:
+        return getRelDataType((DecimalTypeInfo) primitiveTypeInfo);
+      case BOOLEAN:
+        return typeFactory.createSqlType(SqlTypeName.BOOLEAN);
+      case LONG:
+        return typeFactory.createSqlType(SqlTypeName.BIGINT);
+      case FLOAT:
+        return typeFactory.createSqlType(SqlTypeName.FLOAT);
+      case DOUBLE:
+        return typeFactory.createSqlType(SqlTypeName.DOUBLE);
+      case DATE:
+        return typeFactory.createSqlType(SqlTypeName.DATE);
+      case TIMESTAMP:
+        return typeFactory.createSqlType(SqlTypeName.TIMESTAMP);
+      case BINARY:
+        return typeFactory.createSqlType(SqlTypeName.VARBINARY);
+    }
+    throw new RuntimeException(String.format(UNSUPPORTED_HIVE_DATA_TYPE_ERROR_MSG, primitiveCategory));
+  }
+
+  private RelDataType getRelDataType(PrimitiveTypeInfo pTypeInfo, SqlTypeName typeName) {
+    int maxLen = TypeInfoUtils.getCharacterLengthForType(pTypeInfo);
+    RelDataType relDataType = typeFactory.createSqlType(typeName, maxLen);
+    return typeFactory.createTypeWithCharsetAndCollation(relDataType, Util.getDefaultCharset(),
+        SqlCollation.IMPLICIT);
+  }
+
+  private RelDataType getRelDataType(DecimalTypeInfo decimalTypeInfo) {
+    return typeFactory.createSqlType(SqlTypeName.DECIMAL, decimalTypeInfo.precision(), decimalTypeInfo.scale());
+  }
+
+}
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/ColumnListsCache.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/ColumnListsCache.java
index 4420155637c..6a5e1c6d212 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/ColumnListsCache.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/ColumnListsCache.java
@@ -17,15 +17,16 @@
  */
 package org.apache.drill.exec.store.hive;
 
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.drill.shaded.guava.com.google.common.base.Preconditions;
 import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
-import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
-import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Table;
 
-import java.util.List;
-import java.util.Map;
-
 /**
  * The class represents "cache" for partition and table columns.
  * Used to reduce physical plan for Hive tables.
@@ -46,8 +47,8 @@ public ColumnListsCache(Table table) {
   }
 
   public ColumnListsCache() {
-    this.fields = Lists.newArrayList();
-    this.keys = Maps.newHashMap();
+    this.fields = new ArrayList<>();
+    this.keys = new HashMap<>();
   }
 
   /**
@@ -83,14 +84,22 @@ public int addOrGet(List<FieldSchema> columns) {
    * or null if index is negative or greater than fields list size
    */
   public List<FieldSchema> getColumns(int index) {
-    if (index >= 0 && index < fields.size()) {
-      return fields.get(index);
-    } else {
-      return null;
-    }
+   return (index > -1 && index < fields.size()) ? fields.get(index) : null;
+  }
+
+  /**
+   * Safely retrieves Hive table columns from cache.
+   *
+   * @return list of table columns defined in hive
+   */
+  public List<FieldSchema> getTableSchemaColumns() {
+    List<FieldSchema> tableSchemaColumns = getColumns(0);
+    Preconditions.checkNotNull(tableSchemaColumns, "Failed to get columns for Hive table from cache.");
+    return tableSchemaColumns;
   }
 
   public List<List<FieldSchema>> getFields() {
-    return Lists.newArrayList(fields);
+    return new ArrayList<>(fields);
   }
+
 }
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveStoragePlugin.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveStoragePlugin.java
index a65a69ea4e1..9bab1d06ec8 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveStoragePlugin.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/HiveStoragePlugin.java
@@ -27,11 +27,9 @@
 import java.util.function.Function;
 import java.util.stream.Collectors;
 
-import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableSet;
-
-import org.apache.calcite.schema.Schema.TableType;
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.calcite.schema.SchemaPlus;
-
 import org.apache.commons.lang3.StringEscapeUtils;
 import org.apache.drill.common.JSONOptions;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
@@ -52,11 +50,9 @@
 import org.apache.drill.exec.store.StoragePluginOptimizerRule;
 import org.apache.drill.exec.store.dfs.FormatPlugin;
 import org.apache.drill.exec.store.hive.schema.HiveSchemaFactory;
-
-import com.fasterxml.jackson.core.type.TypeReference;
-import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.drill.exec.store.mapr.db.MapRDBFormatPlugin;
 import org.apache.drill.exec.store.mapr.db.MapRDBFormatPluginConfig;
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableSet;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.api.MetaException;
@@ -101,11 +97,6 @@ public HiveScan getPhysicalScan(String userName, JSONOptions selection, List<Sch
   public HiveScan getPhysicalScan(String userName, JSONOptions selection, List<SchemaPath> columns, SessionOptionManager options) throws IOException {
     HiveReadEntry hiveReadEntry = selection.getListWith(new ObjectMapper(), new TypeReference<HiveReadEntry>(){});
     try {
-      if (hiveReadEntry.getJdbcTableType() == TableType.VIEW) {
-        throw new UnsupportedOperationException(
-            "Querying views created in Hive from Drill is not supported in current version.");
-      }
-
       Map<String, String> confProperties = new HashMap<>();
       if (options != null) {
         String value = StringEscapeUtils.unescapeJava(options.getString(ExecConstants.HIVE_CONF_PROPERTIES));
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/schema/DrillHiveTable.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/schema/DrillHiveTable.java
index 8bf4162f7db..5c524355cb1 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/schema/DrillHiveTable.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/schema/DrillHiveTable.java
@@ -17,31 +17,21 @@
  */
 package org.apache.drill.exec.store.hive.schema;
 
-import org.apache.calcite.util.Util;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
 
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.drill.exec.planner.logical.DrillTable;
+import org.apache.drill.exec.planner.types.HiveToRelDataTypeConverter;
 import org.apache.drill.exec.store.hive.HiveReadEntry;
 import org.apache.drill.exec.store.hive.HiveStoragePlugin;
 import org.apache.drill.exec.store.hive.HiveTableWithColumnCache;
-import org.apache.hadoop.hive.metastore.api.FieldSchema;
-import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.MapTypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.calcite.sql.SqlCollation;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-import org.apache.drill.shaded.guava.com.google.common.collect.Lists;
 
-public class DrillHiveTable extends DrillTable{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillHiveTable.class);
+public class DrillHiveTable extends DrillTable {
 
   protected final HiveTableWithColumnCache hiveTable;
 
@@ -52,140 +42,14 @@ public DrillHiveTable(String storageEngineName, HiveStoragePlugin plugin, String
 
   @Override
   public RelDataType getRowType(RelDataTypeFactory typeFactory) {
-    List<RelDataType> typeList = Lists.newArrayList();
-    List<String> fieldNameList = Lists.newArrayList();
-
-    List<FieldSchema> hiveFields = hiveTable.getColumnListsCache().getColumns(0);
-    for(FieldSchema hiveField : hiveFields) {
-      fieldNameList.add(hiveField.getName());
-      typeList.add(getNullableRelDataTypeFromHiveType(
-          typeFactory, TypeInfoUtils.getTypeInfoFromTypeString(hiveField.getType())));
-    }
-
-    for (FieldSchema field : hiveTable.getPartitionKeys()) {
-      fieldNameList.add(field.getName());
-      typeList.add(getNullableRelDataTypeFromHiveType(
-          typeFactory, TypeInfoUtils.getTypeInfoFromTypeString(field.getType())));
-    }
-
-    return typeFactory.createStructType(typeList, fieldNameList);
-  }
-
-  private RelDataType getNullableRelDataTypeFromHiveType(RelDataTypeFactory typeFactory, TypeInfo typeInfo) {
-    RelDataType relDataType = getRelDataTypeFromHiveType(typeFactory, typeInfo);
-    return typeFactory.createTypeWithNullability(relDataType, true);
-  }
-
-  private RelDataType getRelDataTypeFromHivePrimitiveType(RelDataTypeFactory typeFactory, PrimitiveTypeInfo pTypeInfo) {
-    switch(pTypeInfo.getPrimitiveCategory()) {
-      case BOOLEAN:
-        return typeFactory.createSqlType(SqlTypeName.BOOLEAN);
-
-      case BYTE:
-      case SHORT:
-        return typeFactory.createSqlType(SqlTypeName.INTEGER);
-
-      case INT:
-        return typeFactory.createSqlType(SqlTypeName.INTEGER);
-
-      case LONG:
-        return typeFactory.createSqlType(SqlTypeName.BIGINT);
-
-      case FLOAT:
-        return typeFactory.createSqlType(SqlTypeName.FLOAT);
-
-      case DOUBLE:
-        return typeFactory.createSqlType(SqlTypeName.DOUBLE);
-
-      case DATE:
-        return typeFactory.createSqlType(SqlTypeName.DATE);
-
-      case TIMESTAMP:
-        return typeFactory.createSqlType(SqlTypeName.TIMESTAMP);
-
-      case BINARY:
-        return typeFactory.createSqlType(SqlTypeName.VARBINARY);
-
-      case DECIMAL: {
-        DecimalTypeInfo decimalTypeInfo = (DecimalTypeInfo)pTypeInfo;
-        return typeFactory.createSqlType(SqlTypeName.DECIMAL, decimalTypeInfo.precision(), decimalTypeInfo.scale());
-      }
-
-      case STRING:
-      case VARCHAR: {
-        int maxLen = TypeInfoUtils.getCharacterLengthForType(pTypeInfo);
-        return typeFactory.createTypeWithCharsetAndCollation(
-          typeFactory.createSqlType(SqlTypeName.VARCHAR, maxLen), /*input type*/
-          Util.getDefaultCharset(),
-          SqlCollation.IMPLICIT /* TODO: need to decide if implicit is the correct one */
-        );
-      }
-
-      case CHAR: {
-        int maxLen = TypeInfoUtils.getCharacterLengthForType(pTypeInfo);
-        return typeFactory.createTypeWithCharsetAndCollation(
-          typeFactory.createSqlType(SqlTypeName.CHAR, maxLen), /*input type*/
-          Util.getDefaultCharset(),
-          SqlCollation.IMPLICIT
-        );
-      }
-
-      case UNKNOWN:
-      case VOID:
-      default:
-        throwUnsupportedHiveDataTypeError(pTypeInfo.getPrimitiveCategory().toString());
-    }
-
-    return null;
+    HiveToRelDataTypeConverter dataTypeConverter = new HiveToRelDataTypeConverter(typeFactory);
+    final List<String> fieldNames = new ArrayList<>();
+    final List<RelDataType> fieldTypes = Stream.of(hiveTable.getColumnListsCache().getTableSchemaColumns(), hiveTable.getPartitionKeys())
+            .flatMap(Collection::stream)
+            .peek(hiveField -> fieldNames.add(hiveField.getName()))
+            .map(dataTypeConverter::convertToNullableRelDataType)
+            .collect(Collectors.toList());
+    return typeFactory.createStructType(fieldTypes, fieldNames);
   }
 
-  private RelDataType getRelDataTypeFromHiveType(RelDataTypeFactory typeFactory, TypeInfo typeInfo) {
-    switch(typeInfo.getCategory()) {
-      case PRIMITIVE:
-        return getRelDataTypeFromHivePrimitiveType(typeFactory, ((PrimitiveTypeInfo) typeInfo));
-
-      case LIST: {
-        ListTypeInfo listTypeInfo = (ListTypeInfo)typeInfo;
-        RelDataType listElemTypeInfo = getRelDataTypeFromHiveType(typeFactory, listTypeInfo.getListElementTypeInfo());
-        return typeFactory.createArrayType(listElemTypeInfo, -1);
-      }
-
-      case MAP: {
-        MapTypeInfo mapTypeInfo = (MapTypeInfo)typeInfo;
-        RelDataType keyType = getRelDataTypeFromHiveType(typeFactory, mapTypeInfo.getMapKeyTypeInfo());
-        RelDataType valueType = getRelDataTypeFromHiveType(typeFactory, mapTypeInfo.getMapValueTypeInfo());
-        return typeFactory.createMapType(keyType, valueType);
-      }
-
-      case STRUCT: {
-        StructTypeInfo structTypeInfo = (StructTypeInfo)typeInfo;
-        ArrayList<String> fieldNames = structTypeInfo.getAllStructFieldNames();
-        ArrayList<TypeInfo> fieldHiveTypeInfoList = structTypeInfo.getAllStructFieldTypeInfos();
-        List<RelDataType> fieldRelDataTypeList = Lists.newArrayList();
-        for(TypeInfo fieldHiveType : fieldHiveTypeInfoList) {
-          fieldRelDataTypeList.add(getRelDataTypeFromHiveType(typeFactory, fieldHiveType));
-        }
-        return typeFactory.createStructType(fieldRelDataTypeList, fieldNames);
-      }
-
-      case UNION:
-        logger.warn("There is no UNION data type in SQL. Converting it to Sql type OTHER to avoid " +
-            "breaking INFORMATION_SCHEMA queries");
-        return typeFactory.createSqlType(SqlTypeName.OTHER);
-    }
-
-    throwUnsupportedHiveDataTypeError(typeInfo.getCategory().toString());
-    return null;
-  }
-
-  private void throwUnsupportedHiveDataTypeError(String hiveType) {
-    StringBuilder errMsg = new StringBuilder();
-    errMsg.append(String.format("Unsupported Hive data type %s. ", hiveType));
-    errMsg.append(System.getProperty("line.separator"));
-    errMsg.append("Following Hive data types are supported in Drill INFORMATION_SCHEMA: ");
-    errMsg.append("BOOLEAN, BYTE, SHORT, INT, LONG, FLOAT, DOUBLE, DATE, TIMESTAMP, BINARY, DECIMAL, STRING, " +
-        "VARCHAR, CHAR, LIST, MAP, STRUCT and UNION");
-
-    throw new RuntimeException(errMsg.toString());
-  }
 }
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/schema/DrillHiveViewTable.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/schema/DrillHiveViewTable.java
index aedc5f22d72..5a9e92d8253 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/schema/DrillHiveViewTable.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/schema/DrillHiveViewTable.java
@@ -17,26 +17,91 @@
  */
 package org.apache.drill.exec.store.hive.schema;
 
-import org.apache.calcite.schema.Schema.TableType;
+import java.util.Collection;
+import java.util.List;
+import java.util.stream.Stream;
 
-import org.apache.drill.exec.planner.logical.DrillViewInfoProvider;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.sql.type.SqlTypeFactoryImpl;
+import org.apache.drill.exec.dotdrill.View;
+import org.apache.drill.exec.planner.logical.DrillViewTable;
+import org.apache.drill.exec.planner.sql.SchemaUtilites;
+import org.apache.drill.exec.planner.types.DrillRelDataTypeSystem;
+import org.apache.drill.exec.planner.types.HiveToRelDataTypeConverter;
+import org.apache.drill.exec.store.SchemaConfig;
 import org.apache.drill.exec.store.hive.HiveReadEntry;
-import org.apache.drill.exec.store.hive.HiveStoragePlugin;
+import org.apache.drill.exec.store.hive.HiveTableWithColumnCache;
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
 
-public class DrillHiveViewTable extends DrillHiveTable implements DrillViewInfoProvider {
+import static java.util.stream.Collectors.toList;
 
-  public DrillHiveViewTable(String storageEngineName, HiveStoragePlugin plugin, String userName,
-      HiveReadEntry readEntry) {
-    super(storageEngineName, plugin, userName, readEntry);
+/**
+ * DrillViewTable which may be created from Hive view metadata and will work
+ * similar to views defined in Drill.
+ */
+public class DrillHiveViewTable extends DrillViewTable {
+
+  private static final HiveToRelDataTypeConverter DATA_TYPE_CONVERTER = new HiveToRelDataTypeConverter(
+      new SqlTypeFactoryImpl(DrillRelDataTypeSystem.DRILL_REL_DATATYPE_SYSTEM));
+
+  public DrillHiveViewTable(HiveReadEntry entry, List<String> schemaPath,
+                            SchemaConfig schemaConfig,
+                            String user) {
+    super(createView(schemaPath, entry.getTable()), user, schemaConfig.getViewExpansionContext());
   }
 
+  /**
+   * Because tables used by hive views, defined without name
+   * of storage plugin, we're making sure that storage plugin
+   * name will be taken into account for the special case,
+   * when hive storage based authorization is used, and user
+   * can query view, but doesn't have rights to  access underlying
+   * table.
+   *
+   * @param context - to rel conversion context
+   * @param rowType - data type of requested columns
+   * @param workspaceSchemaPath - path to view in drill, for example: ["hive"]
+   * @param tokenSchemaTree - schema created for impersonated user
+   * @return - relational representation of expanded Hive view
+   */
   @Override
-  public TableType getJdbcTableType() {
-    return TableType.VIEW;
+  protected RelNode expandViewForImpersonatedUser(RelOptTable.ToRelContext context, RelDataType rowType,
+                                                  List<String> workspaceSchemaPath, SchemaPlus tokenSchemaTree) {
+    SchemaPlus drillHiveSchema = SchemaUtilites.findSchema(tokenSchemaTree, workspaceSchemaPath);
+    workspaceSchemaPath = ImmutableList.of();
+    return super.expandViewForImpersonatedUser(context, rowType, workspaceSchemaPath, drillHiveSchema);
   }
 
-  @Override
-  public String getViewSql() {
-    return hiveTable.getViewExpandedText();
+  /**
+   * Responsible for creation of View based on Hive view metadata.
+   * Usually such instances created as a result of reading .view.drill files.
+   *
+   * @param schemaPath - path to view in drill, for example: ["hive"]
+   * @param hiveView - hive view metadata
+   * @return - View object for further usage
+   */
+  private static View createView(List<String> schemaPath, HiveTableWithColumnCache hiveView) {
+    List<View.FieldType> viewFields = getViewFieldTypes(hiveView);
+    String viewName = hiveView.getTableName();
+    String viewSql = hiveView.getViewExpandedText();
+    return new View(viewName, viewSql, viewFields, schemaPath);
   }
+
+  /**
+   * Helper method for conversion of hive view fields
+   * to drill view fields
+   *
+   * @param hiveTable - hive view metadata
+   * @return - list of fields for construction of View
+   */
+  private static List<View.FieldType> getViewFieldTypes(HiveTableWithColumnCache hiveTable) {
+    return Stream.of(hiveTable.getColumnListsCache().getTableSchemaColumns(), hiveTable.getPartitionKeys())
+        .flatMap(Collection::stream)
+        .map(hiveField -> new View.FieldType(hiveField.getName(), DATA_TYPE_CONVERTER.convertToNullableRelDataType(hiveField)))
+        .collect(toList());
+  }
+
 }
diff --git a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/schema/HiveSchemaFactory.java b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/schema/HiveSchemaFactory.java
index abb867618e9..402dda26b21 100644
--- a/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/schema/HiveSchemaFactory.java
+++ b/contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/schema/HiveSchemaFactory.java
@@ -25,15 +25,12 @@
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.drill.shaded.guava.com.google.common.cache.CacheBuilder;
-import org.apache.drill.shaded.guava.com.google.common.cache.CacheLoader;
-import org.apache.drill.shaded.guava.com.google.common.cache.LoadingCache;
-import org.apache.drill.shaded.guava.com.google.common.cache.RemovalListener;
 import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.Table;
+import org.apache.drill.common.AutoCloseables;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.exec.ExecConstants;
-import org.apache.drill.exec.planner.logical.DrillTable;
 import org.apache.drill.exec.store.AbstractSchema;
 import org.apache.drill.exec.store.AbstractSchemaFactory;
 import org.apache.drill.exec.store.SchemaConfig;
@@ -41,12 +38,17 @@
 import org.apache.drill.exec.store.hive.HiveReadEntry;
 import org.apache.drill.exec.store.hive.HiveStoragePlugin;
 import org.apache.drill.exec.store.hive.HiveStoragePluginConfig;
-import org.apache.drill.exec.util.ImpersonationUtil;
+import org.apache.drill.shaded.guava.com.google.common.cache.CacheBuilder;
+import org.apache.drill.shaded.guava.com.google.common.cache.CacheLoader;
+import org.apache.drill.shaded.guava.com.google.common.cache.LoadingCache;
+import org.apache.drill.shaded.guava.com.google.common.cache.RemovalListener;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.thrift.TException;
 
+import static org.apache.drill.exec.util.ImpersonationUtil.getProcessUserName;
+
 public class HiveSchemaFactory extends AbstractSchemaFactory {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HiveSchemaFactory.class);
 
@@ -104,14 +106,11 @@ private boolean needToImpersonateReadingData() {
    * Close this schema factory in preparation for retrying. Attempt to close
    * connections, but just ignore any errors.
    */
-
   public void close() {
-    try {
-      processUserMetastoreClient.close();
-    } catch (Exception e) { }
-    try {
-      metaStoreClientLoadingCache.invalidateAll();
-    } catch (Exception e) { }
+    AutoCloseables.closeSilently(
+        processUserMetastoreClient::close,
+        metaStoreClientLoadingCache::invalidateAll
+    );
   }
 
   @Override
@@ -160,7 +159,9 @@ public AbstractSchema getSubSchema(String name) {
       }
     }
 
-    /** Help method to get subschema when we know it exists (already checks the existence) */
+    /**
+     * Helper method to get subschema when we know it exists (already checked the existence)
+     */
     private HiveDatabaseSchema getSubSchemaKnownExists(String name) {
       return new HiveDatabaseSchema(this, name, mClient, schemaConfig);
     }
@@ -208,20 +209,19 @@ public boolean areTableNamesCaseSensitive() {
       return false;
     }
 
-    DrillTable getDrillTable(String dbName, String t) {
+    Table getDrillTable(String dbName, String t) {
       HiveReadEntry entry = getSelectionBaseOnName(dbName, t);
       if (entry == null) {
         return null;
       }
+      final String schemaUser = schemaConfig.getUserName();
+      return TableType.VIEW == entry.getJdbcTableType()
+          ? new DrillHiveViewTable(entry, schemaPath, schemaConfig, getUser(schemaUser, entry.getTable().getOwner()))
+          : new DrillHiveTable(getName(), plugin, getUser(schemaUser, getProcessUserName()), entry);
+    }
 
-      final String userToImpersonate = needToImpersonateReadingData() ? schemaConfig.getUserName() :
-          ImpersonationUtil.getProcessUserName();
-
-      if (entry.getJdbcTableType() == TableType.VIEW) {
-        return new DrillHiveViewTable(getName(), plugin, userToImpersonate, entry);
-      } else {
-        return new DrillHiveTable(getName(), plugin, userToImpersonate, entry);
-      }
+    private String getUser(String impersonated, String notImpersonated) {
+      return needToImpersonateReadingData() ? impersonated : notImpersonated;
     }
 
     HiveReadEntry getSelectionBaseOnName(String dbName, String t) {
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestHiveViewsSupport.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestHiveViewsSupport.java
new file mode 100644
index 00000000000..568a7b02042
--- /dev/null
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestHiveViewsSupport.java
@@ -0,0 +1,233 @@
+/*
+ * 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.drill.exec.hive;
+
+import java.math.BigDecimal;
+
+import org.apache.drill.categories.HiveStorageTest;
+import org.apache.drill.categories.SlowTest;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.expr.fn.impl.DateUtility;
+import org.apache.drill.exec.planner.physical.PlannerSettings;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import static org.junit.Assert.assertEquals;
+
+@Category({SlowTest.class, HiveStorageTest.class})
+public class TestHiveViewsSupport extends HiveTestBase {
+
+  @Test
+  public void selectStarFromView() throws Exception {
+    testBuilder()
+        .sqlQuery("SELECT * FROM hive.hive_view")
+        .unOrdered()
+        .baselineColumns("key", "value")
+        .baselineValues(1, " key_1")
+        .baselineValues(2, " key_2")
+        .baselineValues(3, " key_3")
+        .baselineValues(4, " key_4")
+        .baselineValues(5, " key_5")
+        .go();
+  }
+
+  @Test
+  public void useHiveAndSelectStarFromView() throws Exception {
+    test("USE hive");
+    testBuilder()
+        .sqlQuery("SELECT * FROM hive_view")
+        .unOrdered()
+        .baselineColumns("key", "value")
+        .baselineValues(1, " key_1")
+        .baselineValues(2, " key_2")
+        .baselineValues(3, " key_3")
+        .baselineValues(4, " key_4")
+        .baselineValues(5, " key_5")
+        .go();
+  }
+
+  @Test
+  public void joinViewAndTable() throws Exception {
+    testBuilder()
+        .sqlQuery("SELECT v.key AS key, t.`value` AS val " +
+            "FROM hive.kv t " +
+            "INNER JOIN hive.hive_view v " +
+            "ON v.key = t.key AND t.key=1")
+        .unOrdered()
+        .baselineColumns("key", "val")
+        .baselineValues(1, " key_1")
+        .go();
+  }
+
+  @Test
+  public void nativeParquetScanForView() throws Exception {
+    try {
+      setSessionOption(ExecConstants.HIVE_OPTIMIZE_PARQUET_SCAN_WITH_NATIVE_READER, true);
+      setSessionOption(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY, true);
+
+      String query = "select * from hive.kv_native_view where key > 1";
+
+      int actualRowCount = testSql(query);
+      assertEquals("Expected and actual row count should match", 2, actualRowCount);
+
+      testPlanMatchingPatterns(query,
+          new String[]{"HiveDrillNativeParquetScan", "numFiles=1"}, null);
+
+    } finally {
+      resetSessionOption(ExecConstants.HIVE_OPTIMIZE_PARQUET_SCAN_WITH_NATIVE_READER);
+      resetSessionOption(PlannerSettings.ENABLE_DECIMAL_DATA_TYPE_KEY);
+    }
+
+  }
+
+  @Test
+  public void viewWithAllSupportedDataTypes() throws Exception {
+    testBuilder().sqlQuery("SELECT * FROM hive.readtest_view")
+        .unOrdered()
+        .baselineColumns(
+            "binary_field",
+            "boolean_field",
+            "tinyint_field",
+            "decimal0_field",
+            "decimal9_field",
+            "decimal18_field",
+            "decimal28_field",
+            "decimal38_field",
+            "double_field",
+            "float_field",
+            "int_field",
+            "bigint_field",
+            "smallint_field",
+            "string_field",
+            "varchar_field",
+            "timestamp_field",
+            "date_field",
+            "char_field",
+            // There is a regression in Hive 1.2.1 in binary type partition columns. Disable for now.
+            //"binary_part",
+            "boolean_part",
+            "tinyint_part",
+            "decimal0_part",
+            "decimal9_part",
+            "decimal18_part",
+            "decimal28_part",
+            "decimal38_part",
+            "double_part",
+            "float_part",
+            "int_part",
+            "bigint_part",
+            "smallint_part",
+            "string_part",
+            "varchar_part",
+            "timestamp_part",
+            "date_part",
+            "char_part")
+        .baselineValues(
+            "binaryfield".getBytes(),
+            false,
+            34,
+            new BigDecimal("66"),
+            new BigDecimal("2347.92"),
+            new BigDecimal("2758725827.99990"),
+            new BigDecimal("29375892739852.8"),
+            new BigDecimal("89853749534593985.783"),
+            8.345d,
+            4.67f,
+            123456,
+            234235L,
+            3455,
+            "stringfield",
+            "varcharfield",
+            DateUtility.parseBest("2013-07-05 17:01:00"),
+            DateUtility.parseLocalDate("2013-07-05"),
+            "charfield",
+            // There is a regression in Hive 1.2.1 in binary type partition columns. Disable for now.
+            //"binary",
+            true,
+            64,
+            new BigDecimal("37"),
+            new BigDecimal("36.90"),
+            new BigDecimal("3289379872.94565"),
+            new BigDecimal("39579334534534.4"),
+            new BigDecimal("363945093845093890.900"),
+            8.345d,
+            4.67f,
+            123456,
+            234235L,
+            3455,
+            "string",
+            "varchar",
+            DateUtility.parseBest("2013-07-05 17:01:00"),
+            DateUtility.parseLocalDate("2013-07-05"),
+            "char")
+        .baselineValues( // All fields are null, but partition fields have non-null values
+            null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null,
+            // There is a regression in Hive 1.2.1 in binary type partition columns. Disable for now.
+            //"binary",
+            true,
+            64,
+            new BigDecimal("37"),
+            new BigDecimal("36.90"),
+            new BigDecimal("3289379872.94565"),
+            new BigDecimal("39579334534534.4"),
+            new BigDecimal("363945093845093890.900"),
+            8.345d,
+            4.67f,
+            123456,
+            234235L,
+            3455,
+            "string",
+            "varchar",
+            DateUtility.parseBest("2013-07-05 17:01:00"),
+            DateUtility.parseLocalDate("2013-07-05"),
+            "char")
+        .go();
+  }
+
+  @Test
+  public void viewOverView() throws Exception {
+    testBuilder()
+        .sqlQuery("SELECT * FROM hive.view_over_hive_view")
+        .unOrdered()
+        .baselineColumns("key", "value")
+        .baselineValues(2, " key_2")
+        .baselineValues(3, " key_3")
+        .go();
+  }
+
+  @Test
+  public void materializedViews() throws Exception {
+    testBuilder()
+        .sqlQuery("SELECT * FROM hive.hive_view_m")
+        .unOrdered()
+        .baselineColumns("key", "value")
+        .baselineValues(1, " key_1")
+        .go();
+  }
+
+  @Test
+  public void viewOverTablesInDifferentSchema() throws Exception {
+    testBuilder()
+        .sqlQuery("SELECT dk_key_count FROM hive.db1.two_table_view")
+        .unOrdered()
+        .baselineColumns("dk_key_count")
+        .baselineValues(5L)
+        .go();
+  }
+
+}
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestInfoSchemaOnHiveStorage.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestInfoSchemaOnHiveStorage.java
index f8b467fb964..ccd89181876 100644
--- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestInfoSchemaOnHiveStorage.java
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/hive/TestInfoSchemaOnHiveStorage.java
@@ -42,7 +42,7 @@ public void showTablesFromDb() throws Exception{
         .baselineValues("hive.default", "readtest_parquet")
         .baselineValues("hive.default", "empty_table")
         .baselineValues("hive.default", "infoschematest")
-        .baselineValues("hive.default", "hiveview")
+        .baselineValues("hive.default", "hive_view")
         .baselineValues("hive.default", "kv")
         .baselineValues("hive.default", "kv_parquet")
         .baselineValues("hive.default", "kv_sh")
@@ -51,6 +51,10 @@ public void showTablesFromDb() throws Exception{
         .baselineValues("hive.default", "kv_native")
         .baselineValues("hive.default", "kv_native_ext")
         .baselineValues("hive.default", "sub_dir_table")
+        .baselineValues("hive.default", "readtest_view")
+        .baselineValues("hive.default", "kv_native_view")
+        .baselineValues("hive.default", "hive_view_m")
+        .baselineValues("hive.default", "view_over_hive_view")
         .go();
 
     testBuilder()
@@ -59,6 +63,7 @@ public void showTablesFromDb() throws Exception{
         .baselineColumns("TABLE_SCHEMA", "TABLE_NAME")
         .baselineValues("hive.db1", "kv_db1")
         .baselineValues("hive.db1", "avro")
+        .baselineValues("hive.db1", "two_table_view")
         .go();
 
     testBuilder()
@@ -242,12 +247,13 @@ public void showInfoSchema() throws Exception {
         .baselineColumns("TABLE_CATALOG", "TABLE_SCHEMA", "TABLE_NAME", "TABLE_TYPE")
         .baselineValues("DRILL", "hive.db1", "kv_db1", "TABLE")
         .baselineValues("DRILL", "hive.db1", "avro", "TABLE")
+        .baselineValues("DRILL", "hive.db1", "two_table_view", "VIEW")
         .baselineValues("DRILL", "hive.default", "kv", "TABLE")
         .baselineValues("DRILL", "hive.default", "empty_table", "TABLE")
         .baselineValues("DRILL", "hive.default", "readtest", "TABLE")
         .baselineValues("DRILL", "hive.default", "infoschematest", "TABLE")
         .baselineValues("DRILL", "hive.default", "readtest_parquet", "TABLE")
-        .baselineValues("DRILL", "hive.default", "hiveview", "VIEW")
+        .baselineValues("DRILL", "hive.default", "hive_view", "VIEW")
         .baselineValues("DRILL", "hive.default", "partition_pruning_test", "TABLE")
         .baselineValues("DRILL", "hive.default", "partition_with_few_schemas", "TABLE")
         .baselineValues("DRILL", "hive.default", "kv_parquet", "TABLE")
@@ -256,6 +262,10 @@ public void showInfoSchema() throws Exception {
         .baselineValues("DRILL", "hive.default", "kv_native", "TABLE")
         .baselineValues("DRILL", "hive.default", "kv_native_ext", "TABLE")
         .baselineValues("DRILL", "hive.default", "sub_dir_table", "TABLE")
+        .baselineValues("DRILL", "hive.default", "readtest_view", "VIEW")
+        .baselineValues("DRILL", "hive.default", "kv_native_view", "VIEW")
+        .baselineValues("DRILL", "hive.default", "hive_view_m", "TABLE")
+        .baselineValues("DRILL", "hive.default", "view_over_hive_view", "VIEW")
         .baselineValues("DRILL", "hive.skipper", "kv_text_small", "TABLE")
         .baselineValues("DRILL", "hive.skipper", "kv_text_large", "TABLE")
         .baselineValues("DRILL", "hive.skipper", "kv_incorrect_skip_header", "TABLE")
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/impersonation/hive/BaseTestHiveImpersonation.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/impersonation/hive/BaseTestHiveImpersonation.java
index e361c66053b..53088edbf6d 100644
--- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/impersonation/hive/BaseTestHiveImpersonation.java
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/impersonation/hive/BaseTestHiveImpersonation.java
@@ -27,12 +27,15 @@
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
+import org.apache.hadoop.hive.ql.Driver;
 import org.apache.hadoop.hive.shims.ShimLoader;
 
 import java.io.File;
+import java.io.IOException;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.drill.exec.hive.HiveTestUtilities.executeQuery;
 import static org.apache.drill.exec.store.hive.HiveTestDataGenerator.createFileWithPermissions;
 import static org.apache.hadoop.fs.FileSystem.FS_DEFAULT_NAME_KEY;
 import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.METASTOREURIS;
@@ -49,10 +52,12 @@
   protected static final String studentDef = "CREATE TABLE %s.%s" +
       "(rownum int, name string, age int, gpa float, studentnum bigint) " +
       "ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' STORED AS TEXTFILE";
+
   protected static final String voterDef = "CREATE TABLE %s.%s" +
       "(voter_id int,name varchar(30), age tinyint, registration string, " +
       "contributions double,voterzone smallint,create_time timestamp) " +
       "ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' STORED AS TEXTFILE";
+
   protected static final String partitionStudentDef = "CREATE TABLE %s.%s" +
       "(rownum INT, name STRING, gpa FLOAT, studentnum BIGINT) " +
       "partitioned by (age INT) ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' STORED AS TEXTFILE";
@@ -132,8 +137,8 @@ protected void showTablesHelper(final String db, List<String> expectedTables) th
     testBuilder.go();
   }
 
-  protected void fromInfoSchemaHelper(final String pluginName, final String db, List<String> expectedTables, List<TableType> expectedTableTypes) throws Exception {
-    final String dbQualified = pluginName + "." + db;
+  protected void fromInfoSchemaHelper(final String db, List<String> expectedTables, List<TableType> expectedTableTypes) throws Exception {
+    final String dbQualified = hivePluginName + "." + db;
     final TestBuilder testBuilder = testBuilder()
         .sqlQuery("SELECT TABLE_SCHEMA, TABLE_NAME, TABLE_TYPE \n" +
             "FROM INFORMATION_SCHEMA.`TABLES` \n" +
@@ -157,4 +162,39 @@ public static void stopHiveMetaStore() throws Exception {
     // exit. As each metastore server instance is using its own resources and not sharing it with other metastore
     // server instances this should be ok.
   }
+
+  static void queryView(String viewName) throws Exception {
+    String query = String.format("SELECT rownum FROM %s.tmp.%s ORDER BY rownum LIMIT 1", MINI_DFS_STORAGE_PLUGIN_NAME, viewName);
+    testBuilder()
+        .sqlQuery(query)
+        .unOrdered()
+        .baselineColumns("rownum")
+        .baselineValues(1)
+        .go();
+  }
+
+  static void queryViewNotAuthorized(String viewName) throws Exception {
+    String query = String.format("SELECT rownum FROM %s.tmp.%s ORDER BY rownum LIMIT 1", MINI_DFS_STORAGE_PLUGIN_NAME, viewName);
+    errorMsgTestHelper(query, String.format(
+        "Not authorized to read view [%s] in schema [%s.tmp]", viewName, MINI_DFS_STORAGE_PLUGIN_NAME));
+  }
+
+  static void createTableWithStoragePermissions(final Driver hiveDriver, final String db, final String tbl, final String tblDef,
+                                                final String tblData, final String user, final String group, final short permissions) throws Exception {
+    createTable(hiveDriver, db, tbl, tblDef, tblData);
+    setStoragePermissions(db, tbl, user, group, permissions);
+  }
+
+  static void setStoragePermissions(String db, String tbl, String user, String group, short permissions) throws IOException {
+    final Path p = getWhPathForHiveObject(db, tbl);
+    fs.setPermission(p, new FsPermission(permissions));
+    fs.setOwner(p, user, group);
+  }
+
+  static void createTable(final Driver driver, final String db, final String tbl, final String tblDef,
+                          final String data) throws Exception {
+    executeQuery(driver, String.format(tblDef, db, tbl));
+    executeQuery(driver, String.format("LOAD DATA LOCAL INPATH '%s' INTO TABLE %s.%s", data, db, tbl));
+  }
+
 }
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/impersonation/hive/TestSqlStdBasedAuthorization.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/impersonation/hive/TestSqlStdBasedAuthorization.java
index 08469fc34a7..67c663d5c87 100644
--- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/impersonation/hive/TestSqlStdBasedAuthorization.java
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/impersonation/hive/TestSqlStdBasedAuthorization.java
@@ -30,6 +30,7 @@
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -55,9 +56,16 @@
 
   // Tables in "db_general"
   private static final String g_student_user0 = "student_user0";
+
+  private static final String vw_student_user0 = "vw_student_user0";
+
   private static final String g_voter_role0 = "voter_role0";
+
+  private static final String vw_voter_role0 = "vw_voter_role0";
+
   private static final String g_student_user2 = "student_user2";
 
+  private static final String vw_student_user2 = "vw_student_user2";
 
   // Create a view on "g_student_user0". View is owned by user0:group0 and has permissions 750
   private static final String v_student_u0g0_750 = "v_student_u0g0_750";
@@ -65,12 +73,6 @@
   // Create a view on "v_student_u0g0_750". View is owned by user1:group1 and has permissions 750
   private static final String v_student_u1g1_750 = "v_student_u1g1_750";
 
-  private static final String query_v_student_u0g0_750 = String.format(
-      "SELECT rownum FROM %s.%s.%s ORDER BY rownum LIMIT 1", MINI_DFS_STORAGE_PLUGIN_NAME, "tmp", v_student_u0g0_750);
-
-  private static final String query_v_student_u1g1_750 = String.format(
-      "SELECT rownum FROM %s.%s.%s ORDER BY rownum LIMIT 1", MINI_DFS_STORAGE_PLUGIN_NAME, "tmp", v_student_u1g1_750);
-
   // Role for testing purpose
   private static final String test_role0 = "role0";
 
@@ -110,24 +112,56 @@ private static void setSqlStdBasedAuthorizationInHiveConf() {
     return hiveConfig;
   }
 
+
+  /*
+   * Generating database objects with permissions:
+   * |                                         | org1Users[0] | org1Users[1] | org1Users[2]
+   * ---------------------------------------------------------------------------------------
+   * db_general.g_student_user0                |      +       |      -       |      -       |
+   * db_general.g_voter_role0                  |      -       |      +       |      +       |
+   * db_general.g_student_user2                |      -       |      -       |      +       |
+   * |                                         |              |              |              |
+   * mini_dfs_plugin.tmp.v_student_u0g0_750    |      +       |      +       |      -       |
+   * mini_dfs_plugin.tmp.v_student_u1g1_750    |      -       |      +       |      +       |
+   * |                                         |              |              |              |
+   * db_general.vw_student_user0               |      +       |      -       |      -       |
+   * db_general.vw_voter_role0                 |      -       |      +       |      +       |
+   * db_general.vw_student_user2               |      -       |      -       |      +       |
+   * ---------------------------------------------------------------------------------------
+   */
   private static void generateTestData() throws Exception {
     final SessionState ss = new SessionState(hiveConf);
     SessionState.start(ss);
     final Driver driver = new Driver(hiveConf);
 
     executeQuery(driver, "CREATE DATABASE " + db_general);
-    createTbl(driver, db_general, g_student_user0, studentDef, studentData);
-    createTbl(driver, db_general, g_voter_role0, voterDef, voterData);
-    createTbl(driver, db_general, g_student_user2, studentDef, studentData);
+    createTable(driver, db_general, g_student_user0, studentDef, studentData);
+    createTable(driver, db_general, g_voter_role0, voterDef, voterData);
+    createTable(driver, db_general, g_student_user2, studentDef, studentData);
+
+    createHiveView(driver, db_general, vw_student_user0, g_student_user0);
+    createHiveView(driver, db_general, vw_voter_role0, g_voter_role0);
+    createHiveView(driver, db_general, vw_student_user2, g_student_user2);
 
     executeQuery(driver, "SET ROLE admin");
     executeQuery(driver, "CREATE ROLE " + test_role0);
     executeQuery(driver, "GRANT ROLE " + test_role0 + " TO USER " + org1Users[1]);
     executeQuery(driver, "GRANT ROLE " + test_role0 + " TO USER " + org1Users[2]);
 
-    executeQuery(driver, String.format("GRANT SELECT ON %s.%s TO USER %s", db_general, g_student_user0, org1Users[0]));
-    executeQuery(driver, String.format("GRANT SELECT ON %s.%s TO ROLE %s", db_general, g_voter_role0, test_role0));
-    executeQuery(driver, String.format("GRANT SELECT ON %s.%s TO USER %s", db_general, g_student_user2, org1Users[2]));
+    executeQuery(driver, String.format("GRANT SELECT ON db_general.%s TO USER %s",
+        g_student_user0, org1Users[0]));
+    executeQuery(driver, String.format("GRANT SELECT ON db_general.%s TO USER %s",
+        vw_student_user0, org1Users[0]));
+
+    executeQuery(driver, String.format("GRANT SELECT ON db_general.%s TO ROLE %s",
+        g_voter_role0, test_role0));
+    executeQuery(driver, String.format("GRANT SELECT ON db_general.%s TO ROLE %s",
+        vw_voter_role0, test_role0));
+
+    executeQuery(driver, String.format("GRANT SELECT ON db_general.%s TO USER %s",
+        g_student_user2, org1Users[2]));
+    executeQuery(driver, String.format("GRANT SELECT ON db_general.%s TO USER %s",
+        vw_student_user2, org1Users[2]));
 
     createView(org1Users[0], org1Groups[0], v_student_u0g0_750,
         String.format("SELECT rownum, name, age, studentnum FROM %s.%s.%s",
@@ -137,14 +171,9 @@ private static void generateTestData() throws Exception {
         String.format("SELECT rownum, name, age FROM %s.%s.%s", MINI_DFS_STORAGE_PLUGIN_NAME, "tmp", v_student_u0g0_750));
   }
 
-  private static void createTbl(final Driver driver, final String db, final String tbl, final String tblDef,
-      final String data) throws Exception {
-    executeQuery(driver, String.format(tblDef, db, tbl));
-    executeQuery(driver, String.format("LOAD DATA LOCAL INPATH '%s' INTO TABLE %s.%s", data, db, tbl));
-  }
-
   // Irrespective of each db permissions, all dbs show up in "SHOW SCHEMAS"
   @Test
+  @Ignore //todo: enable after fix of DRILL-6923
   public void showSchemas() throws Exception {
     testBuilder()
         .sqlQuery("SHOW SCHEMAS LIKE 'hive.%'")
@@ -156,31 +185,22 @@ public void showSchemas() throws Exception {
   }
 
   @Test
-  public void showTables_user0() throws Exception {
+  public void user0_showTables() throws Exception {
     updateClient(org1Users[0]);
     showTablesHelper(db_general,
         // Users are expected to see all tables in a database even if they don't have permissions to read from tables.
         ImmutableList.of(
             g_student_user0,
             g_student_user2,
-            g_voter_role0
-        ));
-  }
-
-  @Test
-  public void showTables_user1() throws Exception {
-    updateClient(org1Users[1]);
-    showTablesHelper(db_general,
-        // Users are expected to see all tables in a database even if they don't have permissions to read from tables.
-        ImmutableList.of(
-            g_student_user0,
-            g_student_user2,
-            g_voter_role0
+            g_voter_role0,
+            vw_student_user0,
+            vw_voter_role0,
+            vw_student_user2
         ));
   }
 
   @Test
-  public void select_user0_1() throws Exception {
+  public void user0_allowed_g_student_user0() throws Exception {
     // SELECT on "student_user0" table is granted to user "user0"
     updateClient(org1Users[0]);
     test("USE " + hivePluginName + "." + db_general);
@@ -188,7 +208,12 @@ public void select_user0_1() throws Exception {
   }
 
   @Test
-  public void select_user0_2() throws Exception {
+  public void user0_allowed_vw_student_user0() throws Exception {
+    queryHiveView(org1Users[0], vw_student_user0);
+  }
+
+  @Test
+  public void user0_forbidden_g_voter_role0() throws Exception {
     // SELECT on table "student_user0" is NOT granted to user "user0" directly or indirectly through role "role0" as
     // user "user0" is not part of role "role0"
     updateClient(org1Users[0]);
@@ -199,7 +224,39 @@ public void select_user0_2() throws Exception {
   }
 
   @Test
-  public void select_user1_1() throws Exception {
+  public void user0_forbidden_vw_voter_role0() throws Exception {
+    queryHiveViewNotAuthorized(org1Users[0], vw_voter_role0);
+  }
+
+  @Test
+  public void user0_forbidden_v_student_u1g1_750() throws Exception {
+    updateClient(org1Users[0]);
+    queryViewNotAuthorized(v_student_u1g1_750);
+  }
+
+  @Test
+  public void user0_allowed_v_student_u0g0_750() throws Exception {
+    updateClient(org1Users[0]);
+    queryView(v_student_u0g0_750);
+  }
+
+  @Test
+  public void user1_showTables() throws Exception {
+    updateClient(org1Users[1]);
+    showTablesHelper(db_general,
+        // Users are expected to see all tables in a database even if they don't have permissions to read from tables.
+        ImmutableList.of(
+            g_student_user0,
+            g_student_user2,
+            g_voter_role0,
+            vw_student_user0,
+            vw_voter_role0,
+            vw_student_user2
+        ));
+  }
+
+  @Test
+  public void user1_forbidden_g_student_user0() throws Exception {
     // SELECT on table "student_user0" is NOT granted to user "user1"
     updateClient(org1Users[1]);
     test("USE " + hivePluginName + "." + db_general);
@@ -209,7 +266,12 @@ public void select_user1_1() throws Exception {
   }
 
   @Test
-  public void select_user1_2() throws Exception {
+  public void user1_forbidden_vw_student_user0() throws Exception {
+    queryHiveViewNotAuthorized(org1Users[1], vw_student_user0);
+  }
+
+  @Test
+  public void user1_allowed_g_voter_role0() throws Exception {
     // SELECT on "voter_role0" table is granted to role "role0" and user "user1" is part the role "role0"
     updateClient(org1Users[1]);
     test("USE " + hivePluginName + "." + db_general);
@@ -217,7 +279,12 @@ public void select_user1_2() throws Exception {
   }
 
   @Test
-  public void select_user1_3() throws Exception {
+  public void user1_allowed_vw_voter_role0() throws Exception {
+    queryHiveView(org1Users[1], vw_voter_role0);
+  }
+
+  @Test
+  public void user1_allowed_g_voter_role0_but_forbidden_g_student_user2() throws Exception {
     // SELECT on "voter_role0" table is granted to role "role0" and user "user1" is part the role "role0"
     // SELECT on "student_user2" table is NOT granted to either role "role0" or user "user1"
     updateClient(org1Users[1]);
@@ -229,72 +296,81 @@ public void select_user1_3() throws Exception {
   }
 
   @Test
-  public void select_user2_1() throws Exception {
-    // SELECT on "voter_role0" table is granted to role "role0" and user "user2" is part the role "role0"
-    updateClient(org1Users[2]);
+  public void user1_allowed_vw_voter_role0_but_forbidden_vw_student_user2() throws Exception {
+    // SELECT on "vw_voter_role0" table is granted to role "role0" and user "user1" is part the role "role0"
+    // SELECT on "vw_student_user2" table is NOT granted to either role "role0" or user "user1"
+    updateClient(org1Users[1]);
     test("USE " + hivePluginName + "." + db_general);
-    test(String.format("SELECT * FROM %s ORDER BY name LIMIT 2", g_voter_role0));
+    final String query =
+        String.format("SELECT * FROM %s v JOIN %s s on v.name = s.name limit 2;", vw_voter_role0, vw_student_user2);
+    errorMsgTestHelper(query, "Principal [name=user1_1, type=USER] does not have following privileges for " +
+        "operation QUERY [[SELECT] on Object [type=TABLE_OR_VIEW, name=db_general.vw_student_user2]]");
   }
 
   @Test
-  public void select_user2_2() throws Exception {
-    // SELECT on "student_user2" table is granted to user "user2"
-    updateClient(org1Users[2]);
-    test("USE " + hivePluginName + "." + db_general);
-    test(String.format("SELECT * FROM %s ORDER BY name LIMIT 2", g_student_user2));
+  public void user1_allowed_v_student_u0g0_750() throws Exception {
+    updateClient(org1Users[1]);
+    queryView(v_student_u0g0_750);
+  }
+
+  @Test
+  public void user1_allowed_v_student_u1g1_750() throws Exception {
+    updateClient(org1Users[1]);
+    queryView(v_student_u1g1_750);
   }
 
   @Test
-  public void select_user2_3() throws Exception {
+  public void user2_allowed_g_voter_role0() throws Exception {
     // SELECT on "voter_role0" table is granted to role "role0" and user "user2" is part the role "role0"
-    // SELECT on "student_user2" table is granted to user "user2"
     updateClient(org1Users[2]);
     test("USE " + hivePluginName + "." + db_general);
-    test(String.format("SELECT * FROM %s v JOIN %s s on v.name = s.name limit 2;", g_voter_role0, g_student_user2));
+    test(String.format("SELECT * FROM %s ORDER BY name LIMIT 2", g_voter_role0));
   }
 
-  private static void queryViewHelper(final String queryUser, final String query) throws Exception {
-    updateClient(queryUser);
-    testBuilder()
-        .sqlQuery(query)
-        .unOrdered()
-        .baselineColumns("rownum")
-        .baselineValues(1)
-        .go();
+  @Test
+  public void user2_allowed_vw_voter_role0() throws Exception {
+    queryHiveView(org1Users[2], vw_voter_role0);
   }
 
   @Test
-  public void selectUser0_v_student_u0g0_750() throws Exception {
-    queryViewHelper(org1Users[0], query_v_student_u0g0_750);
+  public void user2_allowed_g_student_user2() throws Exception {
+    // SELECT on "student_user2" table is granted to user "user2"
+    updateClient(org1Users[2]);
+    test("USE " + hivePluginName + "." + db_general);
+    test(String.format("SELECT * FROM %s ORDER BY name LIMIT 2", g_student_user2));
   }
 
   @Test
-  public void selectUser1_v_student_u0g0_750() throws Exception {
-    queryViewHelper(org1Users[1], query_v_student_u0g0_750);
+  public void user2_allowed_vw_student_user2() throws Exception {
+    queryHiveView(org1Users[2], vw_student_user2);
   }
 
   @Test
-  public void selectUser2_v_student_u0g0_750() throws Exception {
+  public void user2_allowed_g_voter_role0_and_g_student_user2() throws Exception {
+    // SELECT on "voter_role0" table is granted to role "role0" and user "user2" is part the role "role0"
+    // SELECT on "student_user2" table is granted to user "user2"
     updateClient(org1Users[2]);
-    errorMsgTestHelper(query_v_student_u0g0_750, String.format(
-        "Not authorized to read view [v_student_u0g0_750] in schema [%s.tmp]", MINI_DFS_STORAGE_PLUGIN_NAME));
+    test("USE " + hivePluginName + "." + db_general);
+    test(String.format("SELECT * FROM %s v JOIN %s s on v.name = s.name limit 2;", g_voter_role0, g_student_user2));
   }
 
   @Test
-  public void selectUser0_v_student_u1g1_750() throws Exception {
-    updateClient(org1Users[0]);
-    errorMsgTestHelper(query_v_student_u1g1_750, String.format(
-        "Not authorized to read view [v_student_u1g1_750] in schema [%s.tmp]", MINI_DFS_STORAGE_PLUGIN_NAME));
+  public void user2_allowed_vw_voter_role0_and_vw_student_user2() throws Exception {
+    updateClient(org1Users[2]);
+    test("USE " + hivePluginName + "." + db_general);
+    test(String.format("SELECT * FROM %s v JOIN %s s on v.name = s.name limit 2;", vw_voter_role0, vw_student_user2));
   }
 
   @Test
-  public void selectUser1_v_student_u1g1_750() throws Exception {
-    queryViewHelper(org1Users[1], query_v_student_u1g1_750);
+  public void user2_forbidden_v_student_u0g0_750() throws Exception {
+    updateClient(org1Users[2]);
+    queryViewNotAuthorized(v_student_u0g0_750);
   }
 
   @Test
-  public void selectUser2_v_student_u1g1_750() throws Exception {
-    queryViewHelper(org1Users[2], query_v_student_u1g1_750);
+  public void user2_allowed_v_student_u1g1_750() throws Exception {
+    updateClient(org1Users[2]);
+    queryView(v_student_u1g1_750);
   }
 
   @AfterClass
@@ -302,4 +378,33 @@ public static void shutdown() throws Exception {
     stopMiniDfsCluster();
     stopHiveMetaStore();
   }
+
+  private static void queryHiveView(String usr, String viewName) throws Exception {
+    String query = String.format("SELECT COUNT(*) AS rownum FROM %s.%s.%s",
+        hivePluginName, db_general, viewName);
+    updateClient(usr);
+    testBuilder()
+        .sqlQuery(query)
+        .unOrdered()
+        .baselineColumns("rownum")
+        .baselineValues(1L)
+        .go();
+  }
+
+  private static void queryHiveViewNotAuthorized(String usr, String viewName) throws Exception {
+    final String query = String.format("SELECT * FROM %s.%s.%s", hivePluginName, db_general, viewName);
+    final String expectedError = String.format("Principal [name=%s, type=USER] does not have following privileges for " +
+            "operation QUERY [[SELECT] on Object [type=TABLE_OR_VIEW, name=db_general.%s]]\n",
+        usr, viewName);
+
+    updateClient(usr);
+    errorMsgTestHelper(query, expectedError);
+  }
+
+  private static void createHiveView(Driver driver, String db, String viewName, String tblName) {
+    String viewFullName = db + "." + viewName;
+    String tblFullName = db + "." + tblName;
+    executeQuery(driver, String.format("CREATE OR REPLACE VIEW %s AS SELECT * FROM %s LIMIT 1", viewFullName, tblFullName));
+  }
+
 }
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/impersonation/hive/TestStorageBasedHiveAuthorization.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/impersonation/hive/TestStorageBasedHiveAuthorization.java
index f4389f8b263..78943281218 100644
--- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/impersonation/hive/TestStorageBasedHiveAuthorization.java
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/impersonation/hive/TestStorageBasedHiveAuthorization.java
@@ -17,13 +17,15 @@
  */
 package org.apache.drill.exec.impersonation.hive;
 
-import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
-import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
 
 import org.apache.calcite.schema.Schema.TableType;
 import org.apache.drill.categories.HiveStorageTest;
-import org.apache.drill.exec.store.dfs.WorkspaceConfig;
 import org.apache.drill.categories.SlowTest;
+import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
+import org.apache.drill.shaded.guava.com.google.common.collect.Maps;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hive.ql.Driver;
@@ -36,11 +38,11 @@
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import java.util.Collections;
-import java.util.Map;
-
+import static java.util.Collections.emptyList;
 import static org.apache.drill.exec.hive.HiveTestUtilities.executeQuery;
+import static org.apache.drill.shaded.guava.com.google.common.collect.Lists.newArrayList;
 import static org.apache.hadoop.fs.FileSystem.FS_DEFAULT_NAME_KEY;
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.DYNAMICPARTITIONINGMODE;
 import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_CBO_ENABLED;
 import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_METASTORE_AUTHENTICATOR_MANAGER;
 import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_METASTORE_AUTHORIZATION_AUTH_READS;
@@ -51,7 +53,6 @@
 import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.METASTORE_EXECUTE_SET_UGI;
 import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.METASTORE_PRE_EVENT_LISTENERS;
 import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.METASTORE_SCHEMA_VERIFICATION;
-import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.DYNAMICPARTITIONINGMODE;
 
 @Category({SlowTest.class, HiveStorageTest.class})
 public class TestStorageBasedHiveAuthorization extends BaseTestHiveImpersonation {
@@ -61,12 +62,12 @@
 
   // Tables in "db_general"
   private static final String g_student_u0_700 = "student_u0_700";
+  private static final String g_vw_g_student_u0_700 = "vw_u0_700_student_u0_700";
   private static final String g_student_u0g0_750 = "student_u0g0_750";
   private static final String g_student_all_755 = "student_all_755";
   private static final String g_voter_u1_700 = "voter_u1_700";
   private static final String g_voter_u2g1_750 = "voter_u2g1_750";
   private static final String g_voter_all_755 = "voter_all_755";
-
   private static final String g_partitioned_student_u0_700 = "partitioned_student_u0_700";
 
   // DB whose warehouse directory has permissions 700 and owned by user0
@@ -75,6 +76,7 @@
   // Tables in "db_u0_only"
   private static final String u0_student_all_755 = "student_all_755";
   private static final String u0_voter_all_755 = "voter_all_755";
+  private static final String u0_vw_voter_all_755 = "vw_voter_all_755";
 
   // DB whose warehouse directory has permissions 750 and owned by user1 and group1
   private static final String db_u1g1_only = "db_u1g1_only";
@@ -91,25 +93,26 @@
   // Create a view on "v_student_u0g0_750". View is owned by user1:group1 and has permissions 750
   private static final String v_student_u1g1_750 = "v_student_u1g1_750";
 
-  private static final String query_v_student_u0g0_750 = String.format(
-      "SELECT rownum FROM %s.%s.%s ORDER BY rownum LIMIT 1", MINI_DFS_STORAGE_PLUGIN_NAME, "tmp", v_student_u0g0_750);
-
-  private static final String query_v_student_u1g1_750 = String.format(
-      "SELECT rownum FROM %s.%s.%s ORDER BY rownum LIMIT 1", MINI_DFS_STORAGE_PLUGIN_NAME, "tmp", v_student_u1g1_750);
-
   // Create a view on "partitioned_student_u0_700". View is owned by user0:group0 and has permissions 750
   private static final String v_partitioned_student_u0g0_750 = "v_partitioned_student_u0g0_750";
 
   // Create a view on "v_partitioned_student_u0g0_750". View is owned by user1:group1 and has permissions 750
   private static final String v_partitioned_student_u1g1_750 = "v_partitioned_student_u1g1_750";
 
-  private static final String query_v_partitioned_student_u0g0_750 = String.format(
-      "SELECT rownum FROM %s.%s.%s ORDER BY rownum LIMIT 1", MINI_DFS_STORAGE_PLUGIN_NAME, "tmp",
-      v_partitioned_student_u0g0_750);
+  // rwx  -   -
+  // 1. Only owning user have read, write and execute rights
+  private static final short _700 = (short) 0700;
+
+  // rwx  r-x -
+  // 1. Owning user have read, write and execute rights
+  // 2. Owning group have read and execute rights
+  private static final short _750 = (short) 0750;
 
-  private static final String query_v_partitioned_student_u1g1_750 = String.format(
-      "SELECT rownum FROM %s.%s.%s ORDER BY rownum LIMIT 1", MINI_DFS_STORAGE_PLUGIN_NAME, "tmp",
-      v_partitioned_student_u1g1_750);
+  // rwx  r-x r-x
+  // 1. Owning user have read, write and execute rights
+  // 2. Owning group have read and execute rights
+  // 3. Others have read and execute rights
+  private static final short _755 = (short) 0755;
 
   @BeforeClass
   public static void setup() throws Exception {
@@ -119,7 +122,7 @@ public static void setup() throws Exception {
     startHiveMetaStore();
     startDrillCluster(true);
     addHiveStoragePlugin(getHivePluginConfig());
-    addMiniDfsBasedStorage(Maps.<String, WorkspaceConfig>newHashMap());
+    addMiniDfsBasedStorage(new HashMap<>());
     generateTestData();
   }
 
@@ -145,6 +148,38 @@ private static void setStorabaseBasedAuthorizationInHiveConf() {
     return hiveConfig;
   }
 
+  /*
+   * User       Groups
+   * user0  |   group0
+   * user1  |   group0, group1
+   * user2  |   group1, group2
+   *
+   * Generating database objects with permissions:
+   * <p>
+   * |                                         | org1Users[0] | org1Users[1] | org1Users[2]
+   * ---------------------------------------------------------------------------------------
+   * db_general                                |      +       |      +       |      +       |
+   * db_general.g_student_u0_700               |      +       |      -       |      -       |
+   * db_general.g_student_u0g0_750             |      +       |      +       |      -       |
+   * db_general.g_student_all_755              |      +       |      +       |      +       |
+   * db_general.g_voter_u1_700                 |      -       |      +       |      -       |
+   * db_general.g_voter_u2g1_750               |      -       |      +       |      +       |
+   * db_general.g_voter_all_755                |      +       |      +       |      +       |
+   * db_general.g_partitioned_student_u0_700   |      +       |      -       |      -       |
+   * db_general.g_vw_g_student_u0_700          |      +       |      -       |      -       |
+   * |                                         |              |              |              |
+   * db_u0_only                                |      +       |      -       |      -       |
+   * db_u0_only.u0_student_all_755             |      +       |      -       |      -       |
+   * db_u0_only.u0_voter_all_755               |      +       |      -       |      -       |
+   * db_u0_only.u0_vw_voter_all_755            |      +       |      -       |      -       |
+   * |                                         |              |              |              |
+   * db_u1g1_only                              |      -       |      +       |      +       |
+   * db_u1g1_only.u1g1_student_all_755         |      -       |      +       |      +       |
+   * db_u1g1_only.u1g1_student_u1_700          |      -       |      +       |      -       |
+   * db_u1g1_only.u1g1_voter_all_755           |      -       |      +       |      +       |
+   * db_u1g1_only.u1g1_voter_u1_700            |      -       |      +       |      -       |
+   * ---------------------------------------------------------------------------------------
+   */
   private static void generateTestData() throws Exception {
 
     // Generate Hive test tables
@@ -153,46 +188,85 @@ private static void generateTestData() throws Exception {
     final Driver driver = new Driver(hiveConf);
 
     executeQuery(driver, "CREATE DATABASE " + db_general);
-
-    createTable(driver,
-        db_general, g_student_u0_700, studentDef, studentData, org1Users[0], org1Groups[0], (short) 0700);
-    createTable(driver,
-        db_general, g_student_u0g0_750, studentDef, studentData, org1Users[0], org1Groups[0], (short) 0750);
-    createTable(driver,
-        db_general, g_student_all_755, studentDef, studentData, org1Users[2], org1Groups[2], (short) 0755);
-    createTable(driver,
-        db_general, g_voter_u1_700, voterDef, voterData, org1Users[1], org1Groups[1], (short) 0700);
-    createTable(driver,
-        db_general, g_voter_u2g1_750, voterDef, voterData, org1Users[2], org1Groups[1], (short) 0750);
-    createTable(driver,
-        db_general, g_voter_all_755, voterDef, voterData, org1Users[1], org1Groups[1], (short) 0755);
+    createTableWithStoragePermissions(driver,
+        db_general, g_student_u0_700,
+        studentDef, studentData,
+        org1Users[0], org1Groups[0],
+        _700);
+    createHiveView(driver, db_general,
+        g_vw_g_student_u0_700, g_student_u0_700);
+
+    createTableWithStoragePermissions(driver,
+        db_general, g_student_u0g0_750,
+        studentDef, studentData,
+        org1Users[0], org1Groups[0],
+        _750);
+    createTableWithStoragePermissions(driver,
+        db_general, g_student_all_755,
+        studentDef, studentData,
+        org1Users[2], org1Groups[2],
+        _755);
+    createTableWithStoragePermissions(driver,
+        db_general, g_voter_u1_700,
+        voterDef, voterData,
+        org1Users[1], org1Groups[1],
+        _700);
+    createTableWithStoragePermissions(driver,
+        db_general, g_voter_u2g1_750,
+        voterDef, voterData,
+        org1Users[2], org1Groups[1],
+        _750);
+    createTableWithStoragePermissions(driver,
+        db_general, g_voter_all_755,
+        voterDef, voterData,
+        org1Users[1], org1Groups[1],
+        _755);
 
     createPartitionedTable(driver,
-        db_general, g_partitioned_student_u0_700, partitionStudentDef,
-        "INSERT OVERWRITE TABLE %s.%s PARTITION(age) SELECT rownum, name, age, gpa, studentnum FROM %s.%s",
-        g_student_all_755, org1Users[0], org1Groups[0], (short) 0700);
+        org1Users[0], org1Groups[0]
+    );
 
-    changeDBPermissions(db_general, (short) 0755, org1Users[0], org1Groups[0]);
+    changeDBPermissions(db_general, _755, org1Users[0], org1Groups[0]);
 
     executeQuery(driver, "CREATE DATABASE " + db_u1g1_only);
 
-    createTable(driver,
-        db_u1g1_only, u1g1_student_all_755, studentDef, studentData, org1Users[1], org1Groups[1], (short) 0755);
-    createTable(driver,
-        db_u1g1_only, u1g1_student_u1_700, studentDef, studentData, org1Users[1], org1Groups[1], (short) 0700);
-    createTable(driver,
-        db_u1g1_only, u1g1_voter_all_755, voterDef, voterData, org1Users[1], org1Groups[1], (short) 0755);
-    createTable(driver,
-        db_u1g1_only, u1g1_voter_u1_700, voterDef, voterData, org1Users[1], org1Groups[1], (short) 0700);
+    createTableWithStoragePermissions(driver,
+        db_u1g1_only, u1g1_student_all_755,
+        studentDef, studentData,
+        org1Users[1], org1Groups[1],
+        _755);
+    createTableWithStoragePermissions(driver,
+        db_u1g1_only, u1g1_student_u1_700,
+        studentDef, studentData,
+        org1Users[1], org1Groups[1],
+        _700);
+    createTableWithStoragePermissions(driver,
+        db_u1g1_only, u1g1_voter_all_755,
+        voterDef, voterData,
+        org1Users[1], org1Groups[1],
+        _755);
+    createTableWithStoragePermissions(driver,
+        db_u1g1_only, u1g1_voter_u1_700,
+        voterDef, voterData,
+        org1Users[1], org1Groups[1],
+        _700);
+
+    changeDBPermissions(db_u1g1_only, _750, org1Users[1], org1Groups[1]);
 
-    changeDBPermissions(db_u1g1_only, (short) 0750, org1Users[1], org1Groups[1]);
 
     executeQuery(driver, "CREATE DATABASE " + db_u0_only);
-
-    createTable(driver, db_u0_only, u0_student_all_755, studentDef, studentData, org1Users[0], org1Groups[0], (short) 0755);
-    createTable(driver, db_u0_only, u0_voter_all_755, voterDef, voterData, org1Users[0], org1Groups[0], (short) 0755);
-
-    changeDBPermissions(db_u0_only, (short) 0700, org1Users[0], org1Groups[0]);
+    createTableWithStoragePermissions(driver,
+        db_u0_only, u0_student_all_755,
+        studentDef, studentData,
+        org1Users[0], org1Groups[0],
+        _755);
+    createTableWithStoragePermissions(driver,
+        db_u0_only, u0_voter_all_755,
+        voterDef, voterData,
+        org1Users[0], org1Groups[0],
+        _755);
+    createHiveView(driver, db_u0_only, u0_vw_voter_all_755, u0_voter_all_755);
+    changeDBPermissions(db_u0_only, _700, org1Users[0], org1Groups[0]);
 
     createView(org1Users[0], org1Groups[0], v_student_u0g0_750,
         String.format("SELECT rownum, name, age, studentnum FROM %s.%s.%s",
@@ -209,33 +283,27 @@ private static void generateTestData() throws Exception {
         String.format("SELECT rownum, name, age FROM %s.%s.%s", MINI_DFS_STORAGE_PLUGIN_NAME, "tmp", v_partitioned_student_u0g0_750));
   }
 
-  private static void createPartitionedTable(final Driver hiveDriver, final String db, final String tbl,
-      final String tblDef, final String loadTblDef, final String loadTbl, final String user, final String group,
-      final short permissions) throws Exception {
-    executeQuery(hiveDriver, String.format(tblDef, db, tbl));
-    executeQuery(hiveDriver, String.format(loadTblDef, db, tbl, db, loadTbl));
-
-    final Path p = getWhPathForHiveObject(db, tbl);
-    fs.setPermission(p, new FsPermission(permissions));
-    fs.setOwner(p, user, group);
-  }
-
-  private static void createTable(final Driver hiveDriver, final String db, final String tbl, final String tblDef,
-      final String tblData, final String user, final String group, final short permissions) throws Exception {
-    executeQuery(hiveDriver, String.format(tblDef, db, tbl));
-    executeQuery(hiveDriver, String.format("LOAD DATA LOCAL INPATH '%s' INTO TABLE %s.%s", tblData, db, tbl));
-    final Path p = getWhPathForHiveObject(db, tbl);
-    fs.setPermission(p, new FsPermission(permissions));
+  private static void createPartitionedTable(final Driver hiveDriver, final String user, final String group) throws Exception {
+    executeQuery(hiveDriver, String.format(partitionStudentDef, db_general, g_partitioned_student_u0_700));
+    executeQuery(hiveDriver, String.format("INSERT OVERWRITE TABLE %s.%s PARTITION(age) SELECT rownum, name, age, gpa, studentnum FROM %s.%s",
+        db_general, g_partitioned_student_u0_700, db_general, g_student_all_755));
+    final Path p = getWhPathForHiveObject(TestStorageBasedHiveAuthorization.db_general, TestStorageBasedHiveAuthorization.g_partitioned_student_u0_700);
+    fs.setPermission(p, new FsPermission(TestStorageBasedHiveAuthorization._700));
     fs.setOwner(p, user, group);
   }
 
-  private static void changeDBPermissions(final String db, final short perm, final String u, final String g)
-      throws Exception {
+  private static void changeDBPermissions(final String db, final short perm, final String u, final String g) throws Exception {
     Path p = getWhPathForHiveObject(db, null);
     fs.setPermission(p, new FsPermission(perm));
     fs.setOwner(p, u, g);
   }
 
+
+  private static void  createHiveView(Driver driver, String db, String viewName, String tableName) throws IOException {
+    executeQuery(driver, String.format("CREATE OR REPLACE VIEW %s.%s AS SELECT * FROM %s.%s LIMIT 1",
+        db, viewName, db, tableName));
+  }
+
   // Irrespective of each db permissions, all dbs show up in "SHOW SCHEMAS"
   @Test
   public void showSchemas() throws Exception {
@@ -251,123 +319,302 @@ public void showSchemas() throws Exception {
   }
 
   /**
-   * "SHOW TABLE" output for a db, should only contain the tables that the user
-   * has access to read. If the user has no read access to the db, the list will be always empty even if the user has
-   * read access to the tables inside the db.
+   * Should only contain the tables that the user
+   * has access to read.
+   *
    * @throws Exception
    */
   @Test
-  public void showTablesUser0() throws Exception {
+  public void user0_db_general_showTables() throws Exception {
     updateClient(org1Users[0]);
+    showTablesHelper(db_general, ImmutableList.of(
+        g_student_u0_700,
+        g_student_u0g0_750,
+        g_student_all_755,
+        g_voter_all_755,
+        g_partitioned_student_u0_700,
+        g_vw_g_student_u0_700
+    ));
+  }
 
-    showTablesHelper(db_general,
-        ImmutableList.of(
-            g_student_u0_700,
-            g_student_u0g0_750,
-            g_student_all_755,
-            g_voter_all_755,
-            g_partitioned_student_u0_700
-        ));
-
-    showTablesHelper(db_u0_only,
-        ImmutableList.of(
-            u0_student_all_755,
-            u0_voter_all_755
-        ));
-
-    showTablesHelper(db_u1g1_only, Collections.<String>emptyList());
+  @Test
+  public void user0_db_u0_only_showTables() throws Exception {
+    updateClient(org1Users[0]);
+    showTablesHelper(db_u0_only, ImmutableList.of(
+        u0_student_all_755,
+        u0_voter_all_755,
+        u0_vw_voter_all_755
+    ));
   }
 
+  /**
+   * If the user has no read access to the db, the list will be always empty even if the user has
+   * read access to the tables inside the db.
+   */
   @Test
-  public void fromInfoSchemaUser0() throws Exception {
+  public void user0_db_u1g1_only_showTables() throws Exception {
     updateClient(org1Users[0]);
+    showTablesHelper(db_u1g1_only, emptyList());
+  }
 
-    fromInfoSchemaHelper(
-        hivePluginName,
-        db_general,
+  @Test
+  public void user0_db_general_infoSchema() throws Exception {
+    updateClient(org1Users[0]);
+    fromInfoSchemaHelper(db_general,
         ImmutableList.of(
             g_student_u0_700,
             g_student_u0g0_750,
             g_student_all_755,
             g_voter_all_755,
-            g_partitioned_student_u0_700
+            g_partitioned_student_u0_700,
+            g_vw_g_student_u0_700
         ),
         ImmutableList.of(
             TableType.TABLE,
             TableType.TABLE,
             TableType.TABLE,
             TableType.TABLE,
-            TableType.TABLE
+            TableType.TABLE,
+            TableType.VIEW
         ));
+  }
 
-    fromInfoSchemaHelper(
-        hivePluginName,
-        db_u0_only,
+  @Test
+  public void user0_db_u0_only_infoSchema() throws Exception {
+    updateClient(org1Users[0]);
+    fromInfoSchemaHelper(db_u0_only,
         ImmutableList.of(
             u0_student_all_755,
-            u0_voter_all_755
+            u0_voter_all_755,
+            u0_vw_voter_all_755
         ),
         ImmutableList.of(
             TableType.TABLE,
-            TableType.TABLE
+            TableType.TABLE,
+            TableType.VIEW
         ));
+  }
 
-    fromInfoSchemaHelper(
-        hivePluginName,
-        db_u1g1_only,
-        Collections.<String>emptyList(),
-        Collections.<TableType>emptyList());
+  @Test
+  public void user0_db_u1g1_only_infoSchema() throws Exception {
+    updateClient(org1Users[0]);
+    fromInfoSchemaHelper(db_u1g1_only, emptyList(), emptyList());
+  }
+
+  /**
+   * user0 is 700 owner
+   */
+  @Test
+  public void user0_allowed_g_student_u0_700() throws Exception {
+    updateClient(org1Users[0]);
+    queryHiveTableOrView(db_general, g_student_u0_700);
   }
 
   @Test
-  public void showTablesUser1() throws Exception {
+  public void user0_allowed_g_vw_u0_700_over_g_student_u0_700() throws Exception {
+    updateClient(org1Users[0]);
+    queryHiveTableOrView(db_general, g_vw_g_student_u0_700);
+  }
+
+  @Test
+  public void user1_forbidden_g_vw_u0_700_over_g_student_u0_700() throws Exception {
     updateClient(org1Users[1]);
+    queryHiveViewFailed(db_general, g_vw_g_student_u0_700);
+  }
 
-    showTablesHelper(db_general,
-        ImmutableList.of(
-            g_student_u0g0_750,
-            g_student_all_755,
-            g_voter_u1_700,
-            g_voter_u2g1_750,
-            g_voter_all_755
-        ));
+  @Test
+  public void user2_forbidden_g_vw_u0_700_over_g_student_u0_700() throws Exception {
+    updateClient(org1Users[2]);
+    queryHiveViewFailed(db_general, g_vw_g_student_u0_700);
+  }
 
-    showTablesHelper(db_u1g1_only,
-        ImmutableList.of(
-            u1g1_student_all_755,
-            u1g1_student_u1_700,
-            u1g1_voter_all_755,
-            u1g1_voter_u1_700
-        ));
+  @Test
+  public void user0_allowed_u0_vw_voter_all_755() throws Exception {
+    updateClient(org1Users[0]);
+    queryHiveTableOrView(db_u0_only, u0_vw_voter_all_755);
+  }
+
+  @Test
+  public void user1_forbidden_u0_vw_voter_all_755() throws Exception {
+    updateClient(org1Users[1]);
+    queryHiveViewFailed(db_u0_only, u0_vw_voter_all_755);
+  }
+
+  @Test
+  public void user2_forbidden_u0_vw_voter_all_755() throws Exception {
+    updateClient(org1Users[2]);
+    queryHiveViewFailed(db_u0_only, u0_vw_voter_all_755);
+  }
+
+  private void queryHiveViewFailed(String db, String viewName) throws Exception {
+    errorMsgTestHelper(
+        String.format("SELECT * FROM hive.%s.%s LIMIT 2", db, viewName),
+        "Failure validating a view your query is dependent upon.");
+  }
+
+  /**
+   * user0 is 750 owner
+   */
+  @Test
+  public void user0_allowed_g_student_u0g0_750() throws Exception {
+    updateClient(org1Users[0]);
+    queryHiveTableOrView(db_general, g_student_u0g0_750);
+  }
+
+  /**
+   * table owned by user2 and group2,
+   * but user0 can access because Others allowed to read and execute
+   */
+  @Test
+  public void user0_allowed_g_student_all_755() throws Exception {
+    updateClient(org1Users[0]);
+    queryHiveTableOrView(db_general, g_student_all_755);
+  }
+
+  /**
+   * user0 can't access because, user1 is 700 owner
+   */
+  @Test
+  public void user0_forbidden_g_voter_u1_700() throws Exception{
+    updateClient(org1Users[0]);
+    queryTableNotFound(db_general, g_voter_u1_700);
+  }
+
+  /**
+   * user0 can't access, because only user2 and group1 members
+   */
+  @Test
+  public void user0_forbidden_g_voter_u2g1_750() throws Exception{
+    updateClient(org1Users[0]);
+    queryTableNotFound(db_general, g_voter_u2g1_750);
+  }
+
+  /**
+   * user0 allowed because others have r-x access. Despite
+   * of user1 and group1 ownership over the table.
+   */
+  @Test
+  public void user0_allowed_g_voter_all_755() throws Exception {
+    updateClient(org1Users[0]);
+    queryHiveTableOrView(db_general, g_voter_all_755);
+  }
 
-    showTablesHelper(db_u0_only, Collections.<String>emptyList());
+  /**
+   * user0 is 755 owner
+   */
+  @Test
+  public void user0_allowed_u0_student_all_755() throws Exception {
+    updateClient(org1Users[0]);
+    queryHiveTableOrView(db_u0_only, u0_student_all_755);
   }
 
+  /**
+   * user0 is 755 owner
+   */
   @Test
-  public void fromInfoSchemaUser1() throws Exception {
+  public void user0_allowed_u0_voter_all_755() throws Exception {
+    updateClient(org1Users[0]);
+    queryHiveTableOrView(db_u0_only, u0_voter_all_755);
+  }
+
+  /**
+   * user0 is 700 owner
+   */
+  @Test
+  public void user0_allowed_g_partitioned_student_u0_700() throws Exception {
+    updateClient(org1Users[0]);
+    queryHiveTableOrView(db_general, g_partitioned_student_u0_700);
+  }
+
+  /**
+   * user0 doesn't have access to database db_u1g1_only
+   */
+  @Test
+  public void user0_forbidden_u1g1_student_all_755() throws Exception {
+    updateClient(org1Users[0]);
+    queryTableNotFound(db_u1g1_only, u1g1_student_all_755);
+  }
+
+  @Test
+  public void user0_allowed_v_student_u0g0_750() throws Exception {
+    updateClient(org1Users[0]);
+    queryView(v_student_u0g0_750);
+  }
+
+  @Test
+  public void user0_forbidden_v_student_u1g1_750() throws Exception {
+    updateClient(org1Users[0]);
+    queryViewNotAuthorized(v_student_u1g1_750);
+  }
+
+  @Test
+  public void user0_allowed_v_partitioned_student_u0g0_750() throws Exception {
+    updateClient(org1Users[0]);
+    queryView(v_partitioned_student_u0g0_750);
+  }
+
+  @Test
+  public void user0_forbidden_v_partitioned_student_u1g1_750() throws Exception {
+    updateClient(org1Users[0]);
+    queryViewNotAuthorized(v_partitioned_student_u1g1_750);
+  }
+
+  @Test
+  public void user1_db_general_showTables() throws Exception {
     updateClient(org1Users[1]);
+    showTablesHelper(db_general, ImmutableList.of(
+        g_student_u0g0_750,
+        g_student_all_755,
+        g_voter_u1_700,
+        g_voter_u2g1_750,
+        g_voter_all_755,
+        g_vw_g_student_u0_700
+    ));
+  }
 
-    fromInfoSchemaHelper(
-        hivePluginName,
-        db_general,
+  @Test
+  public void user1_db_u1g1_only_showTables() throws Exception {
+    updateClient(org1Users[1]);
+    showTablesHelper(db_u1g1_only, ImmutableList.of(
+        u1g1_student_all_755,
+        u1g1_student_u1_700,
+        u1g1_voter_all_755,
+        u1g1_voter_u1_700
+    ));
+  }
+
+  @Test
+  public void user1_db_u0_only_showTables() throws Exception {
+    updateClient(org1Users[1]);
+    showTablesHelper(db_u0_only, newArrayList(u0_vw_voter_all_755));
+  }
+
+  @Test
+  public void user1_db_general_infoSchema() throws Exception {
+    updateClient(org1Users[1]);
+    fromInfoSchemaHelper(db_general,
         ImmutableList.of(
             g_student_u0g0_750,
             g_student_all_755,
             g_voter_u1_700,
             g_voter_u2g1_750,
-            g_voter_all_755
+            g_voter_all_755,
+            g_vw_g_student_u0_700
         ),
         ImmutableList.of(
             TableType.TABLE,
             TableType.TABLE,
             TableType.TABLE,
             TableType.TABLE,
-            TableType.TABLE
+            TableType.TABLE,
+            TableType.VIEW
         ));
+  }
 
-    fromInfoSchemaHelper(
-        hivePluginName,
-        db_u1g1_only,
+  @Test
+  public void user1_db_u1g1_only_infoSchema() throws Exception {
+    updateClient(org1Users[1]);
+    fromInfoSchemaHelper(db_u1g1_only,
         ImmutableList.of(
             u1g1_student_all_755,
             u1g1_student_u1_700,
@@ -380,198 +627,270 @@ public void fromInfoSchemaUser1() throws Exception {
             TableType.TABLE,
             TableType.TABLE
         ));
-
-    fromInfoSchemaHelper(
-        hivePluginName,
-        db_u0_only,
-        Collections.<String>emptyList(),
-        Collections.<TableType>emptyList());
   }
 
   @Test
-  public void showTablesUser2() throws Exception {
-    updateClient(org1Users[2]);
-
-    showTablesHelper(db_general,
-        ImmutableList.of(
-            g_student_all_755,
-            g_voter_u2g1_750,
-            g_voter_all_755
-        ));
+  public void user1_db_u0_only_infoSchema() throws Exception {
+    updateClient(org1Users[1]);
+    fromInfoSchemaHelper(db_u0_only,
+        newArrayList(u0_vw_voter_all_755), newArrayList(TableType.VIEW));
+  }
 
-    showTablesHelper(db_u1g1_only,
-        ImmutableList.of(
-            u1g1_student_all_755,
-            u1g1_voter_all_755
-        ));
+  /**
+   * user1 can't access, because user0 is 700 owner
+   */
+  @Test
+  public void user1_forbidden_g_student_u0_700() throws Exception {
+    updateClient(org1Users[1]);
+    queryTableNotFound(db_general, g_student_u0_700);
+  }
 
-    showTablesHelper(db_u0_only, Collections.<String>emptyList());
+  /**
+   * user1 allowed because he's a member of group0
+   */
+  @Test
+  public void user1_allowed_g_student_u0g0_750() throws Exception {
+    updateClient(org1Users[1]);
+    queryHiveTableOrView(db_general, g_student_u0g0_750);
   }
 
+  /**
+   * user1 allowed because Others have r-x access
+   */
   @Test
-  public void fromInfoSchemaUser2() throws Exception {
-    updateClient(org1Users[2]);
+  public void user1_allowed_g_student_all_755() throws Exception {
+    updateClient(org1Users[1]);
+    queryHiveTableOrView(db_general, g_student_all_755);
+  }
 
-    fromInfoSchemaHelper(
-        hivePluginName,
-        db_general,
-        ImmutableList.of(
-            g_student_all_755,
-            g_voter_u2g1_750,
-            g_voter_all_755
-        ),
-        ImmutableList.of(
-            TableType.TABLE,
-            TableType.TABLE,
-            TableType.TABLE
-        ));
+  /**
+   * user1 is 700 owner
+   */
+  @Test
+  public void user1_allowed_g_voter_u1_700() throws Exception {
+    updateClient(org1Users[1]);
+    queryHiveTableOrView(db_general, g_voter_u1_700);
+  }
 
-    fromInfoSchemaHelper(
-        hivePluginName,
-        db_u1g1_only,
-        ImmutableList.of(
-            u1g1_student_all_755,
-            u1g1_voter_all_755
-        ),
-        ImmutableList.of(
-            TableType.TABLE,
-            TableType.TABLE
-        ));
+  /**
+   * user1 allowed because he's member of group1
+   */
+  @Test
+  public void user1_allowed_g_voter_u2g1_750() throws Exception {
+    updateClient(org1Users[1]);
+    queryHiveTableOrView(db_general, g_voter_u2g1_750);
+  }
 
-    fromInfoSchemaHelper(
-        hivePluginName,
-        db_u0_only,
-        Collections.<String>emptyList(),
-        Collections.<TableType>emptyList());
+  /**
+   * user1 is 755 owner
+   */
+  @Test
+  public void user1_allowed_g_voter_all_755() throws Exception {
+    updateClient(org1Users[1]);
+    queryHiveTableOrView(db_general, g_voter_all_755);
   }
 
-  // Try to read the tables "user0" has access to read in db_general.
+  /**
+   * here access restricted at db level, only user0 can access  db_u0_only
+   */
   @Test
-  public void selectUser0_db_general() throws Exception {
-    updateClient(org1Users[0]);
+  public void user1_forbidden_u0_student_all_755() throws Exception {
+    updateClient(org1Users[1]);
+    queryTableNotFound(db_u0_only, u0_student_all_755);
+  }
 
-    test(String.format("SELECT * FROM hive.%s.%s ORDER BY gpa DESC LIMIT 2", db_general, g_student_u0_700));
-    test(String.format("SELECT * FROM hive.%s.%s ORDER BY gpa DESC LIMIT 2", db_general, g_student_all_755));
-    test(String.format("SELECT * FROM hive.%s.%s ORDER BY name DESC LIMIT 2", db_general, g_voter_all_755));
+  /**
+   * here access restricted at db level, only user0 can access db_u0_only
+   */
+  @Test
+  public void user1_forbidden_u0_voter_all_755() throws Exception {
+    updateClient(org1Users[1]);
+    queryTableNotFound(db_u0_only, u0_voter_all_755);
+  }
 
-    test(String.format("SELECT * FROM hive.%s.%s ORDER BY gpa DESC LIMIT 2", db_general, g_partitioned_student_u0_700));
+  @Test
+  public void user1_allowed_v_student_u0g0_750() throws Exception {
+    updateClient(org1Users[1]);
+    queryView(v_student_u0g0_750);
   }
 
-  // Try to read the table that "user0" has access to read in db_u0_only
   @Test
-  public void selectUser0_db_u0_only() throws Exception {
-    updateClient(org1Users[0]);
+  public void user1_allowed_v_student_u1g1_750() throws Exception {
+    updateClient(org1Users[1]);
+    queryView(v_student_u1g1_750);
+  }
 
-    test(String.format("SELECT * FROM hive.%s.%s ORDER BY gpa DESC LIMIT 2", db_u0_only, u0_student_all_755));
-    test(String.format("SELECT * FROM hive.%s.%s ORDER BY name DESC LIMIT 2", db_u0_only, u0_voter_all_755));
+  @Test
+  public void user1_allowed_v_partitioned_student_u0g0_750() throws Exception {
+    updateClient(org1Users[1]);
+    queryView(v_partitioned_student_u0g0_750);
   }
 
-  // Try to read the tables "user0" has no access to read in db_u1g1_only
   @Test
-  public void selectUser0_db_u1g1_only() throws Exception {
-    updateClient(org1Users[0]);
+  public void user1_allowed_v_partitioned_student_u1g1_750() throws Exception {
+    updateClient(org1Users[1]);
+    queryView(v_partitioned_student_u1g1_750);
+  }
 
-    errorMsgTestHelper(
-        String.format("SELECT * FROM hive.%s.%s ORDER BY gpa DESC LIMIT 2", db_u1g1_only, u1g1_student_all_755),
-        String.format("Object '%s' not found within 'hive.%s'", u1g1_student_all_755, db_u1g1_only));
+  @Test
+  public void user2_db_general_showTables() throws Exception {
+    updateClient(org1Users[2]);
+    showTablesHelper(db_general, ImmutableList.of(
+        g_student_all_755,
+        g_voter_u2g1_750,
+        g_voter_all_755,
+        g_vw_g_student_u0_700
+    ));
   }
 
-  // Try to read the tables "user1" has access to read in db_general.
   @Test
-  public void selectUser1_db_general() throws Exception {
-    updateClient(org1Users[1]);
+  public void user2_db_u1g1_only_showTables() throws Exception {
+    updateClient(org1Users[2]);
+    showTablesHelper(db_u1g1_only, ImmutableList.of(
+        u1g1_student_all_755,
+        u1g1_voter_all_755
+    ));
+  }
 
-    test(String.format("SELECT * FROM hive.%s.%s ORDER BY gpa DESC LIMIT 2", db_general, g_student_u0g0_750));
-    test(String.format("SELECT * FROM hive.%s.%s ORDER BY gpa DESC LIMIT 2", db_general, g_student_all_755));
-    test(String.format("SELECT * FROM hive.%s.%s ORDER BY name DESC LIMIT 2", db_general, g_voter_u2g1_750));
+  @Test
+  public void user2_db_u0_only_showTables() throws Exception {
+    updateClient(org1Users[2]);
+    showTablesHelper(db_u0_only, newArrayList(u0_vw_voter_all_755));
   }
 
-  // Try to read the tables "user1" has no access to read in db_u0_only
   @Test
-  public void selectUser1_db_u0_only() throws Exception {
-    updateClient(org1Users[1]);
+  public void user2_db_general_infoSchema() throws Exception {
+    updateClient(org1Users[2]);
+    fromInfoSchemaHelper(db_general,
+        ImmutableList.of(
+            g_student_all_755,
+            g_voter_u2g1_750,
+            g_voter_all_755,
+            g_vw_g_student_u0_700
+        ),
+        ImmutableList.of(
+            TableType.TABLE,
+            TableType.TABLE,
+            TableType.TABLE,
+            TableType.VIEW
+        ));
+  }
 
-    errorMsgTestHelper(
-        String.format("SELECT * FROM hive.%s.%s ORDER BY gpa DESC LIMIT 2", db_u0_only, u0_student_all_755),
-        String.format("Object '%s' not found within 'hive.%s'", u0_student_all_755, db_u0_only));
+  @Test
+  public void user2_db_u1g1_only_infoSchema() throws Exception {
+    updateClient(org1Users[2]);
+    fromInfoSchemaHelper(db_u1g1_only,
+        ImmutableList.of(
+            u1g1_student_all_755,
+            u1g1_voter_all_755
+        ),
+        ImmutableList.of(
+            TableType.TABLE,
+            TableType.TABLE
+        ));
   }
 
-  private static void queryViewHelper(final String queryUser, final String query) throws Exception {
-    updateClient(queryUser);
-    testBuilder()
-        .sqlQuery(query)
-        .unOrdered()
-        .baselineColumns("rownum")
-        .baselineValues(1)
-        .go();
+  @Test
+  public void user2_db_u0_only_infoSchema() throws Exception {
+    updateClient(org1Users[2]);
+    fromInfoSchemaHelper(db_u0_only, newArrayList(u0_vw_voter_all_755),
+        newArrayList(TableType.VIEW));
   }
 
+  /**
+   * user2 can't access, because user0 is 700 owner
+   */
   @Test
-  public void selectUser0_v_student_u0g0_750() throws Exception {
-    queryViewHelper(org1Users[0], query_v_student_u0g0_750);
+  public void user2_forbidden_g_student_u0_700() throws Exception {
+    updateClient(org1Users[2]);
+    queryTableNotFound(db_general, g_student_u0_700);
   }
 
+  /**
+   * user2 can't access, only user0 and group0 members have access
+   */
   @Test
-  public void selectUser1_v_student_u0g0_750() throws Exception {
-    queryViewHelper(org1Users[1], query_v_student_u0g0_750);
+  public void user2_forbidden_g_student_u0g0_750() throws Exception {
+    updateClient(org1Users[2]);
+    queryTableNotFound(db_general, g_student_u0_700);
   }
 
+  /**
+   * user2 is 755 owner
+   */
   @Test
-  public void selectUser2_v_student_u0g0_750() throws Exception {
+  public void user2_allowed_g_student_all_755() throws Exception {
     updateClient(org1Users[2]);
-    errorMsgTestHelper(query_v_student_u0g0_750, String.format(
-        "Not authorized to read view [v_student_u0g0_750] in schema [%s.tmp]", MINI_DFS_STORAGE_PLUGIN_NAME));
+    queryHiveTableOrView(db_general, g_student_all_755);
   }
 
+  /**
+   * user2 can't access, because user1 is 700 owner
+   */
   @Test
-  public void selectUser0_v_student_u1g1_750() throws Exception {
-    updateClient(org1Users[0]);
-    errorMsgTestHelper(query_v_student_u1g1_750, String.format(
-        "Not authorized to read view [v_student_u1g1_750] in schema [%s.tmp]", MINI_DFS_STORAGE_PLUGIN_NAME));
+  public void user2_forbidden_g_voter_u1_700() throws Exception {
+    updateClient(org1Users[2]);
+    queryTableNotFound(db_general, g_voter_u1_700);
   }
 
+  /**
+   * user2 is 750 owner
+   */
   @Test
-  public void selectUser1_v_student_u1g1_750() throws Exception {
-    queryViewHelper(org1Users[1], query_v_student_u1g1_750);
+  public void user2_allowed_g_voter_u2g1_750() throws Exception {
+    updateClient(org1Users[2]);
+    queryHiveTableOrView(db_general, g_voter_u2g1_750);
   }
 
+  /**
+   * user2 is member of group1
+   */
   @Test
-  public void selectUser2_v_student_u1g1_750() throws Exception {
-    queryViewHelper(org1Users[2], query_v_student_u1g1_750);
+  public void user2_allowed_g_voter_all_755() throws Exception {
+    updateClient(org1Users[2]);
+    queryHiveTableOrView(db_general, g_voter_all_755);
   }
 
+  /**
+   * here access restricted at db level, only user0 can access db_u0_only
+   */
   @Test
-  public void selectUser0_v_partitioned_student_u0g0_750() throws Exception {
-    queryViewHelper(org1Users[0], query_v_partitioned_student_u0g0_750);
+  public void user2_forbidden_u0_student_all_755() throws Exception {
+    updateClient(org1Users[2]);
+    queryTableNotFound(db_u0_only, u0_student_all_755);
   }
 
+  /**
+   * here access restricted at db level, only user0 can access db_u0_only
+   */
   @Test
-  public void selectUser1_v_partitioned_student_u0g0_750() throws Exception {
-    queryViewHelper(org1Users[1], query_v_partitioned_student_u0g0_750);
+  public void user2_forbidden_u0_voter_all_755() throws Exception {
+    updateClient(org1Users[2]);
+    queryTableNotFound(db_u0_only, u0_voter_all_755);
   }
 
   @Test
-  public void selectUser2_v_partitioned_student_u0g0_750() throws Exception {
+  public void user2_forbidden_v_student_u0g0_750() throws Exception {
     updateClient(org1Users[2]);
-    errorMsgTestHelper(query_v_partitioned_student_u0g0_750, String.format(
-        "Not authorized to read view [v_partitioned_student_u0g0_750] in schema [%s.tmp]", MINI_DFS_STORAGE_PLUGIN_NAME));
+    queryViewNotAuthorized(v_student_u0g0_750);
   }
 
   @Test
-  public void selectUser0_v_partitioned_student_u1g1_750() throws Exception {
-    updateClient(org1Users[0]);
-    errorMsgTestHelper(query_v_partitioned_student_u1g1_750, String.format(
-        "Not authorized to read view [v_partitioned_student_u1g1_750] in schema [%s.tmp]", MINI_DFS_STORAGE_PLUGIN_NAME));
+  public void user2_allowed_v_student_u1g1_750() throws Exception {
+    updateClient(org1Users[2]);
+    queryView(v_student_u1g1_750);
   }
 
   @Test
-  public void selectUser1_v_partitioned_student_u1g1_750() throws Exception {
-    queryViewHelper(org1Users[1], query_v_partitioned_student_u1g1_750);
+  public void user2_forbidden_v_partitioned_student_u0g0_750() throws Exception {
+    updateClient(org1Users[2]);
+    queryViewNotAuthorized(v_partitioned_student_u0g0_750);
   }
 
   @Test
-  public void selectUser2_v_partitioned_student_u1g1_750() throws Exception {
-    queryViewHelper(org1Users[2], query_v_partitioned_student_u1g1_750);
+  public void user2_allowed_v_partitioned_student_u1g1_750() throws Exception {
+    updateClient(org1Users[2]);
+    queryView(v_partitioned_student_u1g1_750);
   }
 
   @AfterClass
@@ -579,4 +898,15 @@ public static void shutdown() throws Exception {
     stopMiniDfsCluster();
     stopHiveMetaStore();
   }
+
+  private static void queryHiveTableOrView(String db, String table) throws Exception {
+    test(String.format("SELECT * FROM hive.%s.%s LIMIT 2", db, table));
+  }
+
+  private static void queryTableNotFound(String db, String table) throws Exception {
+    errorMsgTestHelper(
+        String.format("SELECT * FROM hive.%s.%s LIMIT 2", db, table),
+        String.format("Object '%s' not found within 'hive.%s'", table, db));
+  }
+
 }
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/sql/hive/TestViewSupportOnHiveTables.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/sql/hive/TestViewSupportOnHiveTables.java
index eed3faec922..821200d2666 100644
--- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/sql/hive/TestViewSupportOnHiveTables.java
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/sql/hive/TestViewSupportOnHiveTables.java
@@ -103,10 +103,10 @@ public void viewWithSelectFieldsInDef_SelectFieldsInQuery() throws Exception{
   public void testInfoSchemaWithHiveView() throws Exception {
     testBuilder()
         .optionSettingQueriesForTestQuery("USE hive.`default`")
-        .sqlQuery("SELECT * FROM INFORMATION_SCHEMA.VIEWS WHERE TABLE_NAME = 'hiveview'")
+        .sqlQuery("SELECT * FROM INFORMATION_SCHEMA.VIEWS WHERE TABLE_NAME = 'hive_view'")
         .unOrdered()
         .baselineColumns("TABLE_CATALOG", "TABLE_SCHEMA", "TABLE_NAME", "VIEW_DEFINITION")
-        .baselineValues("DRILL", "hive.default", "hiveview", "SELECT `kv`.`key`, `kv`.`value` FROM `default`.`kv`")
+        .baselineValues("DRILL", "hive.default", "hive_view", "SELECT `kv`.`key`, `kv`.`value` FROM `default`.`kv`")
         .go();
   }
 
@@ -116,4 +116,5 @@ public static void cleanupHiveTestData() throws Exception{
       hiveTest.deleteHiveTestPlugin(getDrillbitContext().getStorage());
     }
   }
+
 }
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/store/hive/HiveTestDataGenerator.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/store/hive/HiveTestDataGenerator.java
index 84fa368c8b9..0b9cd36856c 100644
--- a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/store/hive/HiveTestDataGenerator.java
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/store/hive/HiveTestDataGenerator.java
@@ -459,9 +459,6 @@ private void generateTestData() throws Exception {
             "  char_part='char')"
     );
 
-    // create a Hive view to test how its metadata is populated in Drill's INFORMATION_SCHEMA
-    executeQuery(hiveDriver, "CREATE VIEW IF NOT EXISTS hiveview AS SELECT * FROM kv");
-
     executeQuery(hiveDriver, "CREATE TABLE IF NOT EXISTS " +
         "partition_pruning_test_loadtable(a DATE, b TIMESTAMP, c INT, d INT, e INT) " +
         "ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' STORED AS TEXTFILE");
@@ -547,6 +544,14 @@ private void generateTestData() throws Exception {
 
     createSubDirTable(hiveDriver, testDataFile);
 
+    // hive views
+    executeQuery(hiveDriver, "CREATE OR REPLACE VIEW readtest_view AS SELECT * FROM readtest");
+    executeQuery(hiveDriver, "CREATE VIEW IF NOT EXISTS hive_view AS SELECT * FROM kv");
+    executeQuery(hiveDriver, "CREATE OR REPLACE VIEW kv_native_view AS SELECT * FROM kv_native");
+    executeQuery(hiveDriver, "CREATE MATERIALIZED VIEW IF NOT EXISTS hive_view_m AS SELECT * FROM kv WHERE key = 1");
+    executeQuery(hiveDriver, "CREATE OR REPLACE VIEW view_over_hive_view AS SELECT * FROM hive_view WHERE key BETWEEN 2 AND 3");
+    executeQuery(hiveDriver, "CREATE OR REPLACE VIEW db1.two_table_view AS SELECT COUNT(dk.key) dk_key_count FROM db1.avro dk " +
+        "INNER JOIN kv ON kv.key = dk.key");
     ss.close();
   }
 
@@ -696,4 +701,5 @@ private String generateTestDataWithHeadersAndFooters(String tableName, int rowCo
 
     return sb.toString();
   }
-}
\ No newline at end of file
+
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillViewTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillViewTable.java
index 2eb9137efea..a90a135f8a4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillViewTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillViewTable.java
@@ -17,6 +17,9 @@
  */
 package org.apache.drill.exec.planner.logical;
 
+import java.util.List;
+
+import org.apache.calcite.schema.SchemaPlus;
 import org.apache.drill.shaded.guava.com.google.common.collect.ImmutableList;
 import org.apache.calcite.config.CalciteConnectionConfig;
 import org.apache.calcite.schema.Schema.TableType;
@@ -67,7 +70,7 @@ public RelNode toRel(ToRelContext context, RelOptTable relOptTable) {
 
       if (viewExpansionContext.isImpersonationEnabled()) {
         token = viewExpansionContext.reserveViewExpansionToken(viewOwner);
-        rel = context.expandView(rowType, view.getSql(), token.getSchemaTree(), view.getWorkspaceSchemaPath()).rel;
+        rel = expandViewForImpersonatedUser(context, rowType, view.getWorkspaceSchemaPath(), token.getSchemaTree());
       } else {
         rel = context.expandView(rowType, view.getSql(), view.getWorkspaceSchemaPath(), ImmutableList.<String>of()).rel;
       }
@@ -85,6 +88,14 @@ public RelNode toRel(ToRelContext context, RelOptTable relOptTable) {
     }
   }
 
+
+  protected RelNode expandViewForImpersonatedUser(ToRelContext context,
+                                                  RelDataType rowType,
+                                                  List<String> workspaceSchemaPath,
+                                                  SchemaPlus tokenSchemaTree) {
+    return context.expandView(rowType, view.getSql(), tokenSchemaTree, workspaceSchemaPath).rel;
+  }
+
   @Override
   public TableType getJdbcTableType() {
     return TableType.VIEW;
@@ -104,4 +115,5 @@ public boolean rolledUpColumnValidInsideAgg(String column,
   @Override public boolean isRolledUp(String column) {
     return false;
   }
+
 }
diff --git a/logical/pom.xml b/logical/pom.xml
index 468dc179c20..caa126cedb4 100644
--- a/logical/pom.xml
+++ b/logical/pom.xml
@@ -104,9 +104,6 @@
       </plugin>
       <plugin>
         <artifactId>maven-surefire-plugin</artifactId>
-        <configuration>
-          <useSystemClassLoader>false</useSystemClassLoader>
-        </configuration>
       </plugin>
       <plugin>
         <groupId>org.antlr</groupId>


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services