You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sentry.apache.org by sp...@apache.org on 2017/10/20 01:59:06 UTC

sentry git commit: SENTRY-1980: Move the hive-authz2 HMS client filtering implementation into the sentry-binding-hive module (Sergio Pena, reviewed by kalyan kumar kalvagadda, Na Li)

Repository: sentry
Updated Branches:
  refs/heads/master 436787cb6 -> 800584a16


SENTRY-1980: Move the hive-authz2 HMS client filtering implementation into the sentry-binding-hive module (Sergio Pena, reviewed by kalyan kumar kalvagadda, Na Li)


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

Branch: refs/heads/master
Commit: 800584a164d99567dfb75b5f422efe50e690233b
Parents: 436787c
Author: Sergio Pena <se...@cloudera.com>
Authored: Thu Oct 19 20:57:18 2017 -0500
Committer: Sergio Pena <se...@cloudera.com>
Committed: Thu Oct 19 20:57:18 2017 -0500

----------------------------------------------------------------------
 .../hive/authz/DefaultSentryValidator.java      | 472 +++++++++++++++++++
 .../authz/SentryHiveAuthorizationValidator.java |  57 +++
 .../hive/authz/SentryHiveAuthorizerFactory.java |   8 +-
 .../hive/authz/SentryHiveAuthorizerImpl.java    |  66 +--
 .../binding/util/SimpleSemanticAnalyzer.java    | 373 +++++++++++++++
 5 files changed, 918 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/sentry/blob/800584a1/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/DefaultSentryValidator.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/DefaultSentryValidator.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/DefaultSentryValidator.java
new file mode 100644
index 0000000..d1f071e
--- /dev/null
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/DefaultSentryValidator.java
@@ -0,0 +1,472 @@
+/**
+ * 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.sentry.binding.hive.authz;
+
+import static org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_NAME;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Sets;
+import java.security.CodeSource;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.metadata.AuthorizationException;
+import org.apache.hadoop.hive.ql.plan.HiveOperation;
+import org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAccessControlException;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzContext;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzPluginException;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivilegeObjectType;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.sentry.binding.hive.SentryOnFailureHookContext;
+import org.apache.sentry.binding.hive.SentryOnFailureHookContextImpl;
+import org.apache.sentry.binding.hive.authz.HiveAuthzBinding.HiveHook;
+import org.apache.sentry.binding.hive.authz.HiveAuthzPrivileges.HiveOperationScope;
+import org.apache.sentry.binding.hive.conf.HiveAuthzConf;
+import org.apache.sentry.binding.util.SentryAuthorizerUtil;
+import org.apache.sentry.binding.util.SimpleSemanticAnalyzer;
+import org.apache.sentry.core.common.Subject;
+import org.apache.sentry.core.model.db.AccessURI;
+import org.apache.sentry.core.model.db.Column;
+import org.apache.sentry.core.model.db.DBModelAction;
+import org.apache.sentry.core.model.db.DBModelAuthorizable;
+import org.apache.sentry.core.model.db.DBModelAuthorizable.AuthorizableType;
+import org.apache.sentry.core.model.db.Database;
+import org.apache.sentry.core.model.db.Table;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class used to do authorization. Check if current user has privileges to do the operation.
+ */
+public class DefaultSentryValidator extends SentryHiveAuthorizationValidator {
+
+  public static final Logger LOG = LoggerFactory.getLogger(DefaultSentryValidator.class);
+
+  protected HiveConf conf;
+  protected HiveAuthzConf authzConf;
+  protected HiveAuthenticationProvider authenticator;
+
+  public DefaultSentryValidator(HiveConf conf, HiveAuthzConf authzConf,
+      HiveAuthenticationProvider authenticator) throws Exception {
+    initilize(conf, authzConf, authenticator);
+    this.hiveHook = HiveHook.HiveServer2;
+  }
+
+  public DefaultSentryValidator(HiveHook hiveHook, HiveConf conf, HiveAuthzConf authzConf,
+      HiveAuthenticationProvider authenticator) throws Exception {
+    initilize(conf, authzConf, authenticator);
+    this.hiveHook = hiveHook;
+  }
+
+  /**
+   * initialize authenticator and hiveAuthzBinding.
+   */
+  protected void initilize(HiveConf conf, HiveAuthzConf authzConf,
+      HiveAuthenticationProvider authenticator) throws Exception {
+    Preconditions.checkNotNull(conf, "HiveConf cannot be null");
+    Preconditions.checkNotNull(authzConf, "HiveAuthzConf cannot be null");
+    Preconditions.checkNotNull(authenticator, "Hive authenticator provider cannot be null");
+    this.conf = conf;
+    this.authzConf = authzConf;
+    this.authenticator = authenticator;
+  }
+
+  private HiveHook hiveHook;
+
+  // all operations need to extend at DB scope
+  private static final Set<HiveOperation> EX_DB_ALL = Sets.newHashSet(HiveOperation.DROPDATABASE,
+      HiveOperation.CREATETABLE, HiveOperation.IMPORT, HiveOperation.DESCDATABASE,
+      HiveOperation.ALTERTABLE_RENAME, HiveOperation.LOCKDB, HiveOperation.UNLOCKDB);
+  // input operations need to extend at DB scope
+  private static final Set<HiveOperation> EX_DB_INPUT = Sets.newHashSet(HiveOperation.DROPDATABASE,
+      HiveOperation.DESCDATABASE, HiveOperation.ALTERTABLE_RENAME, HiveOperation.LOCKDB,
+      HiveOperation.UNLOCKDB);
+
+  // all operations need to extend at Table scope
+  private static final Set<HiveOperation> EX_TB_ALL = Sets.newHashSet(HiveOperation.DROPTABLE,
+      HiveOperation.DROPVIEW, HiveOperation.DESCTABLE, HiveOperation.SHOW_TBLPROPERTIES,
+      HiveOperation.SHOWINDEXES, HiveOperation.ALTERTABLE_PROPERTIES,
+      HiveOperation.ALTERTABLE_SERDEPROPERTIES, HiveOperation.ALTERTABLE_CLUSTER_SORT,
+      HiveOperation.ALTERTABLE_FILEFORMAT, HiveOperation.ALTERTABLE_TOUCH,
+      HiveOperation.ALTERTABLE_ADDCOLS, HiveOperation.ALTERTABLE_REPLACECOLS,
+      HiveOperation.ALTERTABLE_RENAMEPART, HiveOperation.ALTERTABLE_ARCHIVE,
+      HiveOperation.ALTERTABLE_UNARCHIVE, HiveOperation.ALTERTABLE_SERIALIZER,
+      HiveOperation.ALTERTABLE_MERGEFILES, HiveOperation.ALTERTABLE_SKEWED,
+      HiveOperation.ALTERTABLE_DROPPARTS, HiveOperation.ALTERTABLE_ADDPARTS,
+      HiveOperation.ALTERTABLE_RENAME, HiveOperation.ALTERTABLE_LOCATION,
+      HiveOperation.ALTERVIEW_PROPERTIES, HiveOperation.ALTERPARTITION_FILEFORMAT,
+      HiveOperation.ALTERPARTITION_SERIALIZER, HiveOperation.ALTERPARTITION_MERGEFILES,
+      HiveOperation.ALTERPARTITION_LOCATION, HiveOperation.ALTERTBLPART_SKEWED_LOCATION,
+      HiveOperation.MSCK, HiveOperation.ALTERINDEX_REBUILD, HiveOperation.LOCKTABLE,
+      HiveOperation.UNLOCKTABLE, HiveOperation.SHOWCOLUMNS, HiveOperation.SHOW_TABLESTATUS,
+      HiveOperation.LOAD, HiveOperation.TRUNCATETABLE);
+  // input operations need to extend at Table scope
+  private static final Set<HiveOperation> EX_TB_INPUT = Sets.newHashSet(HiveOperation.DROPTABLE,
+      HiveOperation.DROPVIEW, HiveOperation.SHOW_TBLPROPERTIES, HiveOperation.SHOWINDEXES,
+      HiveOperation.ALTERINDEX_REBUILD, HiveOperation.LOCKTABLE, HiveOperation.UNLOCKTABLE,
+      HiveOperation.SHOW_TABLESTATUS);
+  private static final Set<HiveOperation> META_TB_INPUT = Sets.newHashSet(HiveOperation.DESCTABLE,
+      HiveOperation.SHOWCOLUMNS);
+
+  /**
+   * Check if current user has privileges to perform given operation type hiveOpType on the given
+   * input and output objects
+   *
+   * @param hiveOpType
+   * @param inputHObjs
+   * @param outputHObjs
+   * @param context
+   * @throws
+   */
+  @Override
+  public void checkPrivileges(HiveOperationType hiveOpType, List<HivePrivilegeObject> inputHObjs,
+      List<HivePrivilegeObject> outputHObjs, HiveAuthzContext context)
+      throws HiveAuthzPluginException, HiveAccessControlException {
+    if (LOG.isDebugEnabled()) {
+      String msg =
+          "Checking privileges for operation " + hiveOpType + " by user "
+              + authenticator.getUserName() + " on " + " input objects " + inputHObjs
+              + " and output objects " + outputHObjs + ". Context Info: " + context;
+      LOG.debug(msg);
+    }
+
+    HiveOperation hiveOp = SentryAuthorizerUtil.convert2HiveOperation(hiveOpType.name());
+    HiveAuthzPrivileges stmtAuthPrivileges = null;
+    if (HiveOperation.DESCTABLE.equals(hiveOp) &&
+        !(context.getCommandString().contains("EXTENDED") || context.getCommandString().contains("FORMATTED")) ) {
+      stmtAuthPrivileges = HiveAuthzPrivilegesMap.getHiveAuthzPrivileges(HiveOperation.SHOWCOLUMNS);
+    } else {
+      stmtAuthPrivileges = HiveAuthzPrivilegesMap.getHiveAuthzPrivileges(hiveOp);
+    }
+
+    HiveAuthzBinding hiveAuthzBinding = null;
+    try {
+      hiveAuthzBinding = getAuthzBinding();
+      if (stmtAuthPrivileges == null) {
+        // We don't handle authorizing this statement
+        return;
+      }
+
+      List<List<DBModelAuthorizable>> inputHierarchyList =
+          SentryAuthorizerUtil.convert2SentryPrivilegeList(hiveAuthzBinding.getAuthServer(),
+              inputHObjs);
+      List<List<DBModelAuthorizable>> outputHierarchyList =
+          SentryAuthorizerUtil.convert2SentryPrivilegeList(hiveAuthzBinding.getAuthServer(),
+              outputHObjs);
+
+      // Workaround for metadata queries
+      addExtendHierarchy(hiveOp, stmtAuthPrivileges, inputHierarchyList, outputHierarchyList,
+          context.getCommandString(), hiveAuthzBinding);
+
+      hiveAuthzBinding.authorize(hiveOp, stmtAuthPrivileges,
+          new Subject(authenticator.getUserName()), inputHierarchyList, outputHierarchyList);
+    } catch (AuthorizationException e) {
+      Database db = null;
+      Table tab = null;
+      if (outputHObjs != null) {
+        for (HivePrivilegeObject obj : outputHObjs) {
+          switch (obj.getType()) {
+            case DATABASE:
+              db = new Database(obj.getObjectName());
+              break;
+            case TABLE_OR_VIEW:
+              db = new Database(obj.getDbname());
+              tab = new Table(obj.getObjectName());
+              break;
+            case PARTITION:
+              db = new Database(obj.getDbname());
+              tab = new Table(obj.getObjectName());
+            case LOCAL_URI:
+            case DFS_URI:
+          }
+        }
+      }
+      String permsRequired = "";
+      SentryOnFailureHookContext hookCtx =
+          new SentryOnFailureHookContextImpl(context.getCommandString(), null, null, hiveOp, db,
+              tab, Collections.<AccessURI>emptyList(), null,
+                  authenticator.getUserName(), context.getIpAddress(), e, authzConf);
+      SentryAuthorizerUtil.executeOnFailureHooks(hookCtx, authzConf);
+      for (String perm : hiveAuthzBinding.getLastQueryPrivilegeErrors()) {
+        permsRequired += perm + ";";
+      }
+      SessionState.get().getConf().set(HiveAuthzConf.HIVE_SENTRY_AUTH_ERRORS, permsRequired);
+      String msg =
+          HiveAuthzConf.HIVE_SENTRY_PRIVILEGE_ERROR_MESSAGE
+              + "\n Required privileges for this query: " + permsRequired;
+      throw new HiveAccessControlException(msg, e);
+    } catch (Exception e) {
+      throw new HiveAuthzPluginException(e.getClass()+ ": " + e.getMessage(), e);
+    } finally {
+      if (hiveAuthzBinding != null) {
+        hiveAuthzBinding.close();
+      }
+    }
+
+    if ("true".equalsIgnoreCase(SessionState.get().getConf()
+        .get(HiveAuthzConf.HIVE_SENTRY_MOCK_COMPILATION))) {
+      throw new HiveAccessControlException(HiveAuthzConf.HIVE_SENTRY_MOCK_ERROR
+          + " Mock query compilation aborted. Set " + HiveAuthzConf.HIVE_SENTRY_MOCK_COMPILATION
+          + " to 'false' for normal query processing");
+    }
+  }
+
+  @VisibleForTesting
+  public HiveAuthzBinding getAuthzBinding() throws Exception {
+    return new HiveAuthzBinding(hiveHook, conf, authzConf);
+  }
+
+  private void addExtendHierarchy(HiveOperation hiveOp, HiveAuthzPrivileges stmtAuthPrivileges,
+      List<List<DBModelAuthorizable>> inputHierarchyList,
+      List<List<DBModelAuthorizable>> outputHierarchyList, String command,
+      HiveAuthzBinding hiveAuthzBinding) throws HiveAuthzPluginException,
+      HiveAccessControlException {
+    String currDatabase = null;
+    switch (stmtAuthPrivileges.getOperationScope()) {
+      case SERVER:
+        // validate server level privileges if applicable. Eg create UDF,register jar etc ..
+        List<DBModelAuthorizable> serverHierarchy = new ArrayList<DBModelAuthorizable>();
+        serverHierarchy.add(hiveAuthzBinding.getAuthServer());
+        inputHierarchyList.add(serverHierarchy);
+        break;
+      case DATABASE:
+        // workaround for metadata queries.
+        if (EX_DB_ALL.contains(hiveOp)) {
+          SimpleSemanticAnalyzer analyzer = new SimpleSemanticAnalyzer(hiveOp, command);
+          currDatabase = analyzer.getCurrentDb();
+
+          List<DBModelAuthorizable> externalAuthorizableHierarchy =
+              new ArrayList<DBModelAuthorizable>();
+          externalAuthorizableHierarchy.add(hiveAuthzBinding.getAuthServer());
+          externalAuthorizableHierarchy.add(new Database(currDatabase));
+
+          if (EX_DB_INPUT.contains(hiveOp)) {
+            inputHierarchyList.add(externalAuthorizableHierarchy);
+          } else {
+            outputHierarchyList.add(externalAuthorizableHierarchy);
+          }
+        }
+        break;
+      case TABLE:
+      case COLUMN:
+        // workaround for drop table/view.
+        if (EX_TB_ALL.contains(hiveOp)) {
+          SimpleSemanticAnalyzer analyzer = new SimpleSemanticAnalyzer(hiveOp, command);
+          currDatabase = analyzer.getCurrentDb();
+          String currTable = analyzer.getCurrentTb();
+
+          List<DBModelAuthorizable> externalAuthorizableHierarchy =
+              new ArrayList<DBModelAuthorizable>();
+          externalAuthorizableHierarchy.add(hiveAuthzBinding.getAuthServer());
+          externalAuthorizableHierarchy.add(new Database(currDatabase));
+          externalAuthorizableHierarchy.add(new Table(currTable));
+
+          if (EX_TB_INPUT.contains(hiveOp)) {
+            inputHierarchyList.add(externalAuthorizableHierarchy);
+          } else if (META_TB_INPUT.contains(hiveOp)) {
+            externalAuthorizableHierarchy.add(Column.SOME);
+            inputHierarchyList.add(externalAuthorizableHierarchy);
+          } else {
+            outputHierarchyList.add(externalAuthorizableHierarchy);
+          }
+        }
+        break;
+      case FUNCTION:
+        if (hiveOp.equals(HiveOperation.CREATEFUNCTION)) {
+          SimpleSemanticAnalyzer analyzer = new SimpleSemanticAnalyzer(hiveOp, command);
+          currDatabase = analyzer.getCurrentDb();
+          String udfClassName = analyzer.getCurrentTb();
+          try {
+            CodeSource udfSrc = Class.forName(udfClassName).getProtectionDomain().getCodeSource();
+            if (udfSrc == null) {
+              throw new HiveAuthzPluginException("Could not resolve the jar for UDF class "
+                  + udfClassName);
+            }
+            String udfJar = udfSrc.getLocation().getPath();
+            if (udfJar == null || udfJar.isEmpty()) {
+              throw new HiveAuthzPluginException("Could not find the jar for UDF class "
+                  + udfClassName + "to validate privileges");
+            }
+            AccessURI udfURI = SentryAuthorizerUtil.parseURI(udfSrc.getLocation().toString(), true);
+            List<DBModelAuthorizable> udfUriHierarchy = new ArrayList<DBModelAuthorizable>();
+            udfUriHierarchy.add(hiveAuthzBinding.getAuthServer());
+            udfUriHierarchy.add(udfURI);
+            inputHierarchyList.add(udfUriHierarchy);
+          } catch (Exception e) {
+            throw new HiveAuthzPluginException("Error retrieving udf class", e);
+          }
+        }
+        break;
+      case CONNECT:
+        /*
+         * The 'CONNECT' is an implicit privilege scope currently used for - USE <db> It's allowed
+         * when the user has any privilege on the current database. For application backward
+         * compatibility, we allow (optional) implicit connect permission on 'default' db.
+         */
+        List<DBModelAuthorizable> connectHierarchy = new ArrayList<DBModelAuthorizable>();
+        connectHierarchy.add(hiveAuthzBinding.getAuthServer());
+        if (hiveOp.equals(HiveOperation.SWITCHDATABASE)) {
+          currDatabase = command.split(" ")[1];
+        }
+        // by default allow connect access to default db
+        Table currTbl = Table.ALL;
+        Database currDB = new Database(currDatabase);
+        Column currCol = Column.ALL;
+        if (DEFAULT_DATABASE_NAME.equalsIgnoreCase(currDatabase) && "false"
+            .equalsIgnoreCase(authzConf.get(
+                HiveAuthzConf.AuthzConfVars.AUTHZ_RESTRICT_DEFAULT_DB.getVar(), "false"))) {
+          currDB = Database.ALL;
+          currTbl = Table.SOME;
+        }
+
+        connectHierarchy.add(currDB);
+        connectHierarchy.add(currTbl);
+        connectHierarchy.add(currCol);
+
+        inputHierarchyList.add(connectHierarchy);
+        break;
+    }
+  }
+
+  @Override
+  public List<HivePrivilegeObject> filterListCmdObjects(List<HivePrivilegeObject> listObjs,
+      HiveAuthzContext context) {
+    if (listObjs != null && listObjs.size() >= 1) {
+      HivePrivilegeObjectType pType = listObjs.get(0).getType();
+      HiveAuthzBinding hiveAuthzBinding = null;
+      try {
+        switch (pType) {
+          case DATABASE:
+            hiveAuthzBinding = getAuthzBinding();
+            listObjs = filterShowDatabases(listObjs, authenticator.getUserName(), hiveAuthzBinding);
+            break;
+          case TABLE_OR_VIEW:
+            hiveAuthzBinding = getAuthzBinding();
+            listObjs = filterShowTables(listObjs, authenticator.getUserName(), hiveAuthzBinding);
+            break;
+        }
+      } catch (Exception e) {
+        LOG.warn(e.getMessage(),e);
+      } finally {
+        if (hiveAuthzBinding != null) {
+          hiveAuthzBinding.close();
+        }
+      }
+    }
+    return listObjs;
+  }
+
+  private List<HivePrivilegeObject> filterShowTables(List<HivePrivilegeObject> listObjs,
+      String userName, HiveAuthzBinding hiveAuthzBinding) {
+    List<HivePrivilegeObject> filteredResult = new ArrayList<HivePrivilegeObject>();
+    Subject subject = new Subject(userName);
+    HiveAuthzPrivileges tableMetaDataPrivilege =
+        new HiveAuthzPrivileges.AuthzPrivilegeBuilder()
+            .addInputObjectPriviledge(AuthorizableType.Column,
+                EnumSet.of(DBModelAction.SELECT, DBModelAction.INSERT))
+            .setOperationScope(HiveOperationScope.TABLE)
+            .setOperationType(
+                HiveAuthzPrivileges.HiveOperationType.INFO)
+            .build();
+
+    for (HivePrivilegeObject obj : listObjs) {
+      // if user has privileges on table, add to filtered list, else discard
+      Table table = new Table(obj.getObjectName());
+      Database database;
+      database = new Database(obj.getDbname());
+
+      List<List<DBModelAuthorizable>> inputHierarchy = new ArrayList<List<DBModelAuthorizable>>();
+      List<List<DBModelAuthorizable>> outputHierarchy = new ArrayList<List<DBModelAuthorizable>>();
+      List<DBModelAuthorizable> externalAuthorizableHierarchy =
+          new ArrayList<DBModelAuthorizable>();
+      externalAuthorizableHierarchy.add(hiveAuthzBinding.getAuthServer());
+      externalAuthorizableHierarchy.add(database);
+      externalAuthorizableHierarchy.add(table);
+      externalAuthorizableHierarchy.add(Column.ALL);
+      inputHierarchy.add(externalAuthorizableHierarchy);
+
+      try {
+        hiveAuthzBinding.authorize(HiveOperation.SHOWTABLES, tableMetaDataPrivilege, subject,
+            inputHierarchy, outputHierarchy);
+        filteredResult.add(obj);
+      } catch (AuthorizationException e) {
+        // squash the exception, user doesn't have privileges, so the table is
+        // not added to
+        // filtered list.
+      }
+    }
+    return filteredResult;
+  }
+
+  private List<HivePrivilegeObject> filterShowDatabases(List<HivePrivilegeObject> listObjs,
+      String userName, HiveAuthzBinding hiveAuthzBinding) {
+    List<HivePrivilegeObject> filteredResult = new ArrayList<HivePrivilegeObject>();
+    Subject subject = new Subject(userName);
+    HiveAuthzPrivileges anyPrivilege =
+        new HiveAuthzPrivileges.AuthzPrivilegeBuilder()
+            .addInputObjectPriviledge(
+                AuthorizableType.Column,
+                EnumSet.of(DBModelAction.SELECT, DBModelAction.INSERT, DBModelAction.ALTER,
+                    DBModelAction.CREATE, DBModelAction.DROP, DBModelAction.INDEX,
+                    DBModelAction.LOCK))
+            .setOperationScope(HiveOperationScope.CONNECT)
+            .setOperationType(
+                HiveAuthzPrivileges.HiveOperationType.QUERY)
+            .build();
+
+    for (HivePrivilegeObject obj : listObjs) {
+      // if user has privileges on database, add to filtered list, else discard
+      Database database = null;
+
+      // if default is not restricted, continue
+      if (DEFAULT_DATABASE_NAME.equalsIgnoreCase(obj.getObjectName())
+          && "false".equalsIgnoreCase(hiveAuthzBinding.getAuthzConf().get(
+              HiveAuthzConf.AuthzConfVars.AUTHZ_RESTRICT_DEFAULT_DB.getVar(), "false"))) {
+        filteredResult.add(obj);
+        continue;
+      }
+
+      database = new Database(obj.getObjectName());
+
+      List<List<DBModelAuthorizable>> inputHierarchy = new ArrayList<List<DBModelAuthorizable>>();
+      List<List<DBModelAuthorizable>> outputHierarchy = new ArrayList<List<DBModelAuthorizable>>();
+      List<DBModelAuthorizable> externalAuthorizableHierarchy =
+          new ArrayList<DBModelAuthorizable>();
+      externalAuthorizableHierarchy.add(hiveAuthzBinding.getAuthServer());
+      externalAuthorizableHierarchy.add(database);
+      externalAuthorizableHierarchy.add(Table.ALL);
+      externalAuthorizableHierarchy.add(Column.ALL);
+      inputHierarchy.add(externalAuthorizableHierarchy);
+
+      try {
+        hiveAuthzBinding.authorize(HiveOperation.SHOWDATABASES, anyPrivilege, subject,
+            inputHierarchy, outputHierarchy);
+        filteredResult.add(obj);
+      } catch (AuthorizationException e) {
+        // squash the exception, user doesn't have privileges, so the table is
+        // not added to
+        // filtered list.
+      }
+    }
+    return filteredResult;
+  }
+}

http://git-wip-us.apache.org/repos/asf/sentry/blob/800584a1/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/SentryHiveAuthorizationValidator.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/SentryHiveAuthorizationValidator.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/SentryHiveAuthorizationValidator.java
new file mode 100644
index 0000000..8e42aaf
--- /dev/null
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/SentryHiveAuthorizationValidator.java
@@ -0,0 +1,57 @@
+/**
+ * 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.sentry.binding.hive.authz;
+
+import java.util.List;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAccessControlException;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthorizationValidator;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzContext;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzPluginException;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject;
+
+/**
+ * This class used to do authorization validate. Check if current user has privileges to do the
+ * operation and filter the select results.
+ */
+public abstract class SentryHiveAuthorizationValidator implements HiveAuthorizationValidator {
+
+  /**
+   * Check if current user has privileges to perform given operation type hiveOpType on the given
+   * input and output objects.
+   *
+   * @param hiveOpType
+   * @param inputHObjs
+   * @param outputHObjs
+   * @param context
+   * @throws HiveAuthzPluginException, HiveAccessControlException
+   */
+  @Override
+  public abstract void checkPrivileges(HiveOperationType hiveOpType,
+      List<HivePrivilegeObject> inputHObjs, List<HivePrivilegeObject> outputHObjs,
+      HiveAuthzContext context) throws HiveAuthzPluginException, HiveAccessControlException;
+
+
+  /**
+   * Filter the select results according current user's permission. remove the object which current
+   * user do not have any privilege on it.
+   *
+   * @param listObjs
+   * @param context
+   */
+  @Override
+  public abstract List<HivePrivilegeObject> filterListCmdObjects(
+      List<HivePrivilegeObject> listObjs, HiveAuthzContext context);
+}

http://git-wip-us.apache.org/repos/asf/sentry/blob/800584a1/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/SentryHiveAuthorizerFactory.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/SentryHiveAuthorizerFactory.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/SentryHiveAuthorizerFactory.java
index 76a757a..db8e428 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/SentryHiveAuthorizerFactory.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/SentryHiveAuthorizerFactory.java
@@ -47,14 +47,20 @@ public class SentryHiveAuthorizerFactory implements HiveAuthorizerFactory {
     HiveAuthzSessionContext sessionContext = applyTestSettings(ctx, conf);
 
     SentryHiveAccessController accessController;
+    SentryHiveAuthorizationValidator authValidator;
     try {
       accessController =
           new DefaultSentryAccessController(conf, authzConf, hiveAuthenticator, sessionContext);
+
+      authValidator =
+          new DefaultSentryValidator(conf, authzConf, hiveAuthenticator);
     } catch (Exception e) {
       throw new HiveAuthzPluginException(e);
     }
 
-    return new SentryHiveAuthorizerImpl(accessController);
+
+
+    return new SentryHiveAuthorizerImpl(accessController, authValidator);
   }
 
   private HiveAuthzSessionContext applyTestSettings(HiveAuthzSessionContext ctx, HiveConf conf) {

http://git-wip-us.apache.org/repos/asf/sentry/blob/800584a1/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/SentryHiveAuthorizerImpl.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/SentryHiveAuthorizerImpl.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/SentryHiveAuthorizerImpl.java
index 4aff76f..1596bce 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/SentryHiveAuthorizerImpl.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/SentryHiveAuthorizerImpl.java
@@ -16,7 +16,6 @@
  */
 package org.apache.sentry.binding.hive.authz;
 
-import java.util.ArrayList;
 import java.util.List;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.exec.SentryHivePrivilegeObjectDesc;
@@ -36,7 +35,6 @@ import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObje
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivilegeObjectType;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveRoleGrant;
 import org.apache.sentry.binding.hive.SentryHivePrivilegeObject;
-import org.apache.sentry.binding.metastore.SentryMetaStoreFilterHook;
 
 /**
  * This is a HiveAuthorizer implementation, and it is used by HiveServer2 to check privileges
@@ -49,14 +47,14 @@ import org.apache.sentry.binding.metastore.SentryMetaStoreFilterHook;
  */
 public class SentryHiveAuthorizerImpl extends AbstractHiveAuthorizer {
   private SentryHiveAccessController accessController;
+  private SentryHiveAuthorizationValidator authValidator;
   static private HiveAuthorizationTranslator hiveTranslator =
       new SentryHiveAuthorizationTranslator();
 
-  private SentryMetaStoreFilterHook filterHook;
-
-  public SentryHiveAuthorizerImpl(SentryHiveAccessController accessController) {
+  public SentryHiveAuthorizerImpl(SentryHiveAccessController accessController,
+      SentryHiveAuthorizationValidator authValidator) {
     this.accessController = accessController;
-    this.filterHook = new SentryMetaStoreFilterHook(null);
+    this.authValidator = authValidator;
   }
 
   @Override
@@ -123,24 +121,16 @@ public class SentryHiveAuthorizerImpl extends AbstractHiveAuthorizer {
   public void checkPrivileges(HiveOperationType hiveOpType, List<HivePrivilegeObject> inputsHObjs,
       List<HivePrivilegeObject> outputHObjs, HiveAuthzContext context)
       throws HiveAuthzPluginException, HiveAccessControlException {
-    // Nothing to do there. Privileges are checked on the Semantic hooks
+    // The privileges for a query are checked on the Semantic hooks (HiveAuthzBindingHook) instead
+    // because of lack of information that Hive pass as parameters on this method.
+    // TODO: This will be fixed as part of SENTRY-1957
+    // authValidator.checkPrivileges(hiveOpType, inputsHObjs, outputHObjs, context);
   }
 
   @Override
   public List<HivePrivilegeObject> filterListCmdObjects(List<HivePrivilegeObject> listObjs,
       HiveAuthzContext context) throws HiveAuthzPluginException, HiveAccessControlException {
-    if (listObjs == null || listObjs.size() == 0) {
-      return listObjs;
-    }
-
-    switch (listObjs.get(0).getType()) {
-      case DATABASE:
-        return filterDbs(listObjs);
-      case TABLE_OR_VIEW:
-        return filterTables(listObjs);
-      default:
-        return listObjs;
-    }
+    return authValidator.filterListCmdObjects(listObjs, context);
   }
 
   @Override
@@ -175,44 +165,6 @@ public class SentryHiveAuthorizerImpl extends AbstractHiveAuthorizer {
     return hiveTranslator;
   }
 
-  private List<HivePrivilegeObject> filterDbs(List<HivePrivilegeObject> listObjs) {
-    List<String> dbList = new ArrayList<>(listObjs.size());
-    for (HivePrivilegeObject o : listObjs) {
-      dbList.add(o.getDbname());
-    }
-
-    List<String> filterDbList = filterHook.filterDatabases(dbList);
-    List<HivePrivilegeObject> filterObjs = new ArrayList<>(filterDbList.size());
-    for (String db : filterDbList) {
-      filterObjs.add(new HivePrivilegeObject(HivePrivilegeObjectType.DATABASE, db, db));
-    }
-
-    return filterObjs;
-  }
-
-  private List<HivePrivilegeObject> filterTables(List<HivePrivilegeObject> listObjs) {
-    if (listObjs == null || listObjs.size() == 0) {
-      return listObjs;
-    }
-
-    List<String> tableList = new ArrayList<>(listObjs.size());
-    for (HivePrivilegeObject o : listObjs) {
-      tableList.add(o.getObjectName());
-    }
-
-    String db = listObjs.get(0).getDbname();
-
-    List<String> filterTableList =
-        filterHook.filterTableNames(db, tableList);
-
-    List<HivePrivilegeObject> filterObjs = new ArrayList<>(filterTableList.size());
-    for (String table : filterTableList) {
-      filterObjs.add(new HivePrivilegeObject(HivePrivilegeObjectType.TABLE_OR_VIEW, db, table));
-    }
-
-    return filterObjs;
-  }
-
   protected static HivePrivilegeObjectType getPrivObjectType(
       SentryHivePrivilegeObjectDesc privSubjectDesc) {
     if (privSubjectDesc.getObject() == null) {

http://git-wip-us.apache.org/repos/asf/sentry/blob/800584a1/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/util/SimpleSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/util/SimpleSemanticAnalyzer.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/util/SimpleSemanticAnalyzer.java
new file mode 100644
index 0000000..09fcd38
--- /dev/null
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/util/SimpleSemanticAnalyzer.java
@@ -0,0 +1,373 @@
+/**
+ * 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.sentry.binding.util;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.hadoop.hive.ql.plan.HiveOperation;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzPluginException;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.sentry.core.model.db.Table;
+
+/**
+ * Currently hive complier doesn't create read/write entities for some operations, e.g. create
+ * table, drop table. This class is a simple semantic analyzer using regex, it is a workaround
+ * approach to extract db_name and tb_name from those operations.
+ *
+ * TODO: This will be removed as part of SENTRY-1957
+ */
+public class SimpleSemanticAnalyzer {
+  private String currentDb;
+  private String currentTb;
+
+  /**
+   * CREATE [TEMPORARY] [EXTERNAL] TABLE [IF NOT EXISTS] [db_name.]table_name ...
+   */
+  private static final String CREATE_TABLE_REGEX = "^(CREATE)\\s+" + "(TEMPORARY\\s+)?"
+      + "(EXTERNAL\\s+)?" + "TABLE\\s+" + "(IF\\s+NOT\\s+EXISTS\\s+)?" + "([A-Za-z0-9._]+)";
+
+  /**
+   * DROP (DATABASE|SCHEMA) [IF EXISTS] database_name [RESTRICT|CASCADE];
+   */
+  private static final String DROP_DB_REGEX = "^DROP\\s+" + "(DATABASE|SCHEMA)\\s+"
+      + "(IF\\s+EXISTS\\s+)?" + "([A-Za-z0-9_]+)";
+
+  /**
+   * DROP TABLE [IF EXISTS] table_name;
+   */
+  private static final String DROP_TABLE_REGEX = "^DROP\\s+" + "TABLE\\s+" + "(IF\\s+EXISTS\\s+)?"
+      + "([A-Za-z0-9._]+)";
+
+  /**
+   * DROP VIEW [IF EXISTS] view_name;
+   */
+  private static final String DROP_VIEW_REGEX = "^DROP\\s+" + "VIEW\\s+" + "(IF\\s+EXISTS\\s+)?"
+      + "([A-Za-z0-9_].+)";
+
+  /**
+   * DESCRIBE DATABASE|SCHEMA [EXTENDED] db_name;
+   */
+  private static final String DESCRIBE_DB_REGEX = "^DESCRIBE\\s+" + "(DATABASE|SCHEMA)\\s+"
+      + "(EXTENDED\\s+)?" + "([A-Za-z0-9_]+)";
+
+  /**
+   * DESCRIBE [EXTENDED|FORMATTED] [db_name.]table_name[.col_name ( [.field_name] | [.'$elem$'] |
+   * [.'$key$'] | [.'$value$'] )* ];
+   */
+  private static final String DESCRIBE_TABLE_REGEX = "^DESCRIBE\\s+"
+      + "((EXTENDED|FORMATTED)\\s+)?" + "([A-Za-z0-9._]+)";
+
+  /**
+   * SHOW [FORMATTED] (INDEX|INDEXES) ON table_with_index [(FROM|IN) db_name];
+   */
+  private static final String SHOW_INDEX_REGEX = "^SHOW\\s+" + "(FORMATTED\\s+)?"
+      + "(INDEX|INDEXES)\\s+" + "ON\\s+" + "([A-Za-z0-9._]+)\\s*"
+      + "((FROM|IN)\\s+([A-Za-z0-9_]+))?";
+
+  /**
+   * SHOW TBLPROPERTIES tblname;
+   */
+  private static final String SHOW_TBLPROPERTIES_REGEX = "^SHOW\\s+" + "TBLPROPERTIES\\s+"
+      + "([A-Za-z0-9._]+)";
+
+  /**
+   * ALTER TABLE table_name ...
+   */
+  private static final String ALTER_TABLE_REGEX = "^ALTER\\s+" + "TABLE\\s+" + "([A-Za-z0-9._]+)";
+
+  /**
+   * ALTER VIEW view_name ...
+   */
+  private static final String ALTER_VIEW_REGEX = "^ALTER\\s+" + "VIEW\\s+" + "([A-Za-z0-9._]+)";
+
+  /**
+   * MSCK REPAIR TABLE table_name;
+   */
+  private static final String MSCK_REGEX = "^MSCK\\s+" + "REPAIR\\s" + "TABLE\\s"
+      + "([A-Za-z0-9._]+)";
+
+  /**
+   * ALTER INDEX index_name ON table_name [PARTITION partition_spec] REBUILD;
+   */
+  private static final String ALTER_INDEX_REGEX = "^ALTER\\s+" + "INDEX\\s+"
+      + "([A-Za-z0-9_]+)\\s+" + "ON\\s" + "([A-Za-z0-9._]+)";
+
+  /**
+   * CREATE FUNCTION [db_name.]function_name AS class_name [USING JAR|FILE|ARCHIVE 'file_uri' [,
+   * JAR|FILE|ARCHIVE 'file_uri'] ];
+   */
+  private static final String CREATE_FUNCTION_REGEX = "^CREATE\\s+" + "(TEMPORARY\\s+)?"
+      + "FUNCTION\\s+" + "([A-Za-z0-9._]+)\\s+" + "AS\\s" + "([A-Za-z0-9._']+)";
+
+  /**
+   * SHOW COLUMNS FROM table_name
+   */
+  private static final String SHOWCOLUMNS = "^SHOW\\s+" + "COLUMNS\\s+" + "(FROM|IN)\\s+"
+      + "([A-Za-z0-9._]+)";
+
+  private static final String SHOW_TABLESTATUS = "^SHOW\\s+" + "TABLE\\s+" + "EXTENDED\\s+" + "IN\\s+"
+      + "([A-Za-z0-9._]+)";
+
+  private static final String LOAD = "^LOAD\\s+" + "DATA\\s+" + "(LOCAL\\s+)?" + "INPATH\\s+"
+      + "([A-Za-z0-9._':///-]+)" +"\\s" + "(OVERWRITE\\s+)?" + "INTO\\s" + "TABLE\\s" + "([A-Za-z0-9._]+)";
+
+  /**
+   * LOCK DATABASE dbname;
+   */
+  private static final String LOCKDB = "^LOCK\\s+" + "DATABASE\\s+" + "([A-Za-z0-9._]+)";
+
+  /**
+   * UNLOCK DATABASE dbname;
+   */
+  private static final String UNLOCKDB = "^UNLOCK\\s+" + "DATABASE\\s+" + "([A-Za-z0-9._]+)";
+
+  /**
+   * LOCK TABLE tblname;
+   */
+  private static final String LOCKTABLE = "^LOCK\\s+" + "TABLE\\s+" + "([A-Za-z0-9._]+)";
+
+  /**
+   * UNLOCK TABLE tblname;
+   */
+  private static final String UNLOCKTABLE = "^UNLOCK\\s+" + "TABLE\\s+" + "([A-Za-z0-9._]+)";
+
+  /**
+   * TRUNCATE TABLE tblname;
+   */
+  private static final String TRUNCATETABLE = "^TRUNCATE\\s+" + "TABLE\\s+" + "([A-Za-z0-9._]+)";
+
+  private static Map<HiveOperation, String> OP_REGEX_MAP = new HashMap<HiveOperation, String>();
+  static {
+    // database metadata
+    OP_REGEX_MAP.put(HiveOperation.DROPDATABASE, DROP_DB_REGEX);
+    OP_REGEX_MAP.put(HiveOperation.DESCDATABASE, DESCRIBE_DB_REGEX);
+
+    // table metadata
+    OP_REGEX_MAP.put(HiveOperation.CREATETABLE, CREATE_TABLE_REGEX);
+    OP_REGEX_MAP.put(HiveOperation.DROPTABLE, DROP_TABLE_REGEX);
+    OP_REGEX_MAP.put(HiveOperation.DROPVIEW, DROP_VIEW_REGEX);
+    OP_REGEX_MAP.put(HiveOperation.DESCTABLE, DESCRIBE_TABLE_REGEX);
+    OP_REGEX_MAP.put(HiveOperation.SHOW_TBLPROPERTIES, SHOW_TBLPROPERTIES_REGEX);
+    OP_REGEX_MAP.put(HiveOperation.ALTERTABLE_PROPERTIES, ALTER_TABLE_REGEX);
+    OP_REGEX_MAP.put(HiveOperation.ALTERTABLE_SERDEPROPERTIES, ALTER_TABLE_REGEX);
+    OP_REGEX_MAP.put(HiveOperation.ALTERTABLE_CLUSTER_SORT, ALTER_TABLE_REGEX);
+    OP_REGEX_MAP.put(HiveOperation.ALTERTABLE_FILEFORMAT, ALTER_TABLE_REGEX);
+    OP_REGEX_MAP.put(HiveOperation.ALTERTABLE_TOUCH, ALTER_TABLE_REGEX);
+    OP_REGEX_MAP.put(HiveOperation.ALTERTABLE_RENAMECOL, ALTER_TABLE_REGEX);
+    OP_REGEX_MAP.put(HiveOperation.ALTERTABLE_ADDCOLS, ALTER_TABLE_REGEX);
+    OP_REGEX_MAP.put(HiveOperation.ALTERTABLE_REPLACECOLS, ALTER_TABLE_REGEX);
+    OP_REGEX_MAP.put(HiveOperation.ALTERTABLE_RENAMEPART, ALTER_TABLE_REGEX);
+    OP_REGEX_MAP.put(HiveOperation.ALTERTABLE_ARCHIVE, ALTER_TABLE_REGEX);
+    OP_REGEX_MAP.put(HiveOperation.ALTERTABLE_UNARCHIVE, ALTER_TABLE_REGEX);
+    OP_REGEX_MAP.put(HiveOperation.ALTERTABLE_SERIALIZER, ALTER_TABLE_REGEX);
+    OP_REGEX_MAP.put(HiveOperation.ALTERTABLE_MERGEFILES, ALTER_TABLE_REGEX);
+    OP_REGEX_MAP.put(HiveOperation.ALTERTABLE_SKEWED, ALTER_TABLE_REGEX);
+    OP_REGEX_MAP.put(HiveOperation.ALTERTABLE_DROPPARTS, ALTER_TABLE_REGEX);
+    OP_REGEX_MAP.put(HiveOperation.ALTERTABLE_ADDPARTS, ALTER_TABLE_REGEX);
+    OP_REGEX_MAP.put(HiveOperation.ALTERTABLE_RENAME, ALTER_TABLE_REGEX);
+    OP_REGEX_MAP.put(HiveOperation.ALTERTABLE_LOCATION, ALTER_TABLE_REGEX);
+    OP_REGEX_MAP.put(HiveOperation.ALTERPARTITION_FILEFORMAT, ALTER_TABLE_REGEX);
+    OP_REGEX_MAP.put(HiveOperation.ALTERPARTITION_SERDEPROPERTIES, ALTER_TABLE_REGEX);
+    OP_REGEX_MAP.put(HiveOperation.ALTERPARTITION_SERIALIZER, ALTER_TABLE_REGEX);
+    OP_REGEX_MAP.put(HiveOperation.ALTERPARTITION_MERGEFILES, ALTER_TABLE_REGEX);
+    OP_REGEX_MAP.put(HiveOperation.ALTERPARTITION_LOCATION, ALTER_TABLE_REGEX);
+    OP_REGEX_MAP.put(HiveOperation.ALTERTBLPART_SKEWED_LOCATION, ALTER_TABLE_REGEX);
+    OP_REGEX_MAP.put(HiveOperation.ALTERVIEW_PROPERTIES, ALTER_VIEW_REGEX);
+    OP_REGEX_MAP.put(HiveOperation.MSCK, MSCK_REGEX);
+    OP_REGEX_MAP.put(HiveOperation.ALTERINDEX_REBUILD, ALTER_INDEX_REGEX);
+    OP_REGEX_MAP.put(HiveOperation.ALTERINDEX_PROPS, ALTER_INDEX_REGEX);
+    OP_REGEX_MAP.put(HiveOperation.LOCKDB, LOCKDB);
+    OP_REGEX_MAP.put(HiveOperation.UNLOCKDB, UNLOCKDB);
+    OP_REGEX_MAP.put(HiveOperation.LOCKTABLE, LOCKTABLE);
+    OP_REGEX_MAP.put(HiveOperation.UNLOCKTABLE, UNLOCKTABLE);
+    OP_REGEX_MAP.put(HiveOperation.SHOWCOLUMNS, SHOWCOLUMNS);
+    OP_REGEX_MAP.put(HiveOperation.SHOW_TABLESTATUS, SHOW_TABLESTATUS);
+    OP_REGEX_MAP.put(HiveOperation.TRUNCATETABLE, TRUNCATETABLE);
+  }
+
+  public SimpleSemanticAnalyzer(HiveOperation hiveOp, String cmd) throws HiveAuthzPluginException {
+    currentDb = SessionState.get().getCurrentDatabase();
+    parse(hiveOp, cmd);
+  }
+
+  private void parse(HiveOperation hiveOp, String cmd) throws HiveAuthzPluginException {
+    switch (hiveOp) {
+      case DROPDATABASE:
+      case DESCDATABASE:
+      case LOCKDB:
+      case UNLOCKDB:
+        parseDbMeta(cmd, OP_REGEX_MAP.get(hiveOp));
+        break;
+      case DESCTABLE:
+      case CREATETABLE:
+      case DROPTABLE:
+      case DROPVIEW:
+      case SHOW_TBLPROPERTIES:
+        // alter table
+      case ALTERTABLE_PROPERTIES:
+      case ALTERTABLE_SERDEPROPERTIES:
+      case ALTERTABLE_CLUSTER_SORT:
+      case ALTERTABLE_FILEFORMAT:
+      case ALTERTABLE_TOUCH:
+      case ALTERTABLE_RENAMECOL:
+      case ALTERTABLE_ADDCOLS:
+      case ALTERTABLE_REPLACECOLS:
+      case ALTERTABLE_RENAMEPART:
+      case ALTERTABLE_ARCHIVE:
+      case ALTERTABLE_UNARCHIVE:
+      case ALTERTABLE_SERIALIZER:
+      case ALTERTABLE_MERGEFILES:
+      case ALTERTABLE_SKEWED:
+      case ALTERTABLE_DROPPARTS:
+      case ALTERTABLE_ADDPARTS:
+      case ALTERTABLE_RENAME:
+      case ALTERTABLE_LOCATION:
+        // alter view
+      case ALTERVIEW_PROPERTIES:
+        // alter partition
+      case ALTERPARTITION_FILEFORMAT:
+      case ALTERPARTITION_SERDEPROPERTIES:
+      case ALTERPARTITION_SERIALIZER:
+      case ALTERPARTITION_MERGEFILES:
+      case ALTERPARTITION_LOCATION:
+      case ALTERTBLPART_SKEWED_LOCATION:
+        // MSCK
+      case MSCK:
+        // alter index
+      case ALTERINDEX_REBUILD:
+      case ALTERINDEX_PROPS:
+      case LOCKTABLE:
+      case UNLOCKTABLE:
+      case SHOWCOLUMNS:
+      case TRUNCATETABLE:
+        parseTableMeta(cmd, OP_REGEX_MAP.get(hiveOp));
+        break;
+      case SHOWINDEXES:
+        parseShowIndex(cmd, SHOW_INDEX_REGEX);
+        break;
+      case CREATEFUNCTION:
+        parseFunction(cmd, CREATE_FUNCTION_REGEX);
+        break;
+      case SHOW_TABLESTATUS:
+        parseTableExtend(cmd, SHOW_TABLESTATUS);
+        break;
+      case LOAD:
+        parseLoadTable(cmd, LOAD);
+        break;
+      default:
+        break;
+    }
+  }
+
+  private void parseLoadTable(String cmd, String load) throws HiveAuthzPluginException {
+    Pattern pattern = Pattern.compile(load, Pattern.CASE_INSENSITIVE);
+    Matcher matcher = pattern.matcher(cmd);
+    if (matcher.find()) {
+      String tbName = matcher.group(matcher.groupCount());
+      extractDbAndTb(tbName.trim());
+    } else {
+      throw new HiveAuthzPluginException("this command " + cmd + " does not match the table meta grammar");
+    }
+  }
+
+  private void parseTableExtend(String cmd, String showTablestatus) throws HiveAuthzPluginException {
+    Pattern pattern = Pattern.compile(showTablestatus, Pattern.CASE_INSENSITIVE);
+    Matcher matcher = pattern.matcher(cmd);
+    if (matcher.find()) {
+      String dbName = matcher.group(matcher.groupCount());
+      currentDb = dbName;
+      currentTb = Table.SOME.getName();
+    } else {
+      throw new HiveAuthzPluginException("this command " + cmd + " does not match the table meta grammar");
+    }
+  }
+
+  private void extractDbAndTb(String tableName) {
+    if (tableName.contains(".")) {
+      String[] tb = tableName.split("\\.");
+      currentDb = tb[0];
+      currentTb = tb[1];
+    } else {
+      currentDb = SessionState.get().getCurrentDatabase();
+      currentTb = tableName;
+    }
+  }
+
+  private void parseDbMeta(String cmd, String regex) throws HiveAuthzPluginException {
+    Pattern pattern = Pattern.compile(regex, Pattern.CASE_INSENSITIVE);
+    Matcher matcher = pattern.matcher(cmd);
+    if (matcher.find()) {
+      currentDb = matcher.group(matcher.groupCount());
+    } else {
+      throw new HiveAuthzPluginException("this command " + cmd
+          + " does not match the database meta grammar");
+    }
+  }
+
+  private void parseTableMeta(String cmd, String regex) throws HiveAuthzPluginException {
+    Pattern pattern = Pattern.compile(regex, Pattern.CASE_INSENSITIVE);
+    Matcher matcher = pattern.matcher(cmd);
+    if (matcher.find()) {
+      String tbName = matcher.group(matcher.groupCount());
+      extractDbAndTb(tbName.trim());
+    } else {
+      throw new HiveAuthzPluginException("this command " + cmd + " does not match the table meta grammar");
+    }
+  }
+
+  private void parseShowIndex(String cmd, String regex) throws HiveAuthzPluginException {
+    Pattern pattern = Pattern.compile(regex, Pattern.CASE_INSENSITIVE);
+    Matcher matcher = pattern.matcher(cmd);
+    if (matcher.find()) {
+      String dbName = matcher.group(matcher.groupCount());
+      String tbName = matcher.group(3);
+      if (dbName != null) {
+        currentDb = dbName;
+        currentTb = tbName;
+      } else {
+        extractDbAndTb(tbName);
+      }
+    } else {
+      throw new HiveAuthzPluginException("this command " + cmd + " does not match the show index grammar");
+    }
+  }
+
+  private void parseFunction(String cmd, String regex) throws HiveAuthzPluginException {
+    Pattern pattern = Pattern.compile(regex, Pattern.CASE_INSENSITIVE);
+    Matcher matcher = pattern.matcher(cmd);
+    if (matcher.find()) {
+      String udfClass = matcher.group(matcher.groupCount());
+      if (udfClass.contains("'")) {
+        currentTb = udfClass.split("'")[1];
+      } else {
+        currentTb = udfClass;
+      }
+    } else {
+      throw new HiveAuthzPluginException("this command " + cmd
+          + " does not match the create function grammar");
+    }
+  }
+
+  public String getCurrentDb() {
+    return currentDb;
+  }
+
+  public String getCurrentTb() {
+    return currentTb;
+  }
+
+}