You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by th...@apache.org on 2014/01/17 22:24:12 UTC

svn commit: r1559255 - in /hive/trunk/ql/src: java/org/apache/hadoop/hive/ql/security/authorization/plugin/ test/org/apache/hadoop/hive/ql/security/authorization/ test/org/apache/hadoop/hive/ql/security/authorization/plugin/

Author: thejas
Date: Fri Jan 17 21:24:11 2014
New Revision: 1559255

URL: http://svn.apache.org/r1559255
Log:
HIVE-5928 : Add a hive authorization plugin api that does not assume privileges needed - 2 (Thejas Nair reviewed by Brock Noland)

Added:
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/DefaultHiveAuthorizerFactory.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAccessController.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizationValidator.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizer.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizerFactory.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizerImpl.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveMetastoreClientFactory.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveMetastoreClientFactoryImpl.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HivePrincipal.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HivePrivilege.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HivePrivilegeObject.java
    hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/
    hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/
    hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveOperationType.java

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/DefaultHiveAuthorizerFactory.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/DefaultHiveAuthorizerFactory.java?rev=1559255&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/DefaultHiveAuthorizerFactory.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/DefaultHiveAuthorizerFactory.java Fri Jan 17 21:24:11 2014
@@ -0,0 +1,30 @@
+/**
+ * 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.hadoop.hive.ql.security.authorization.plugin;
+
+import org.apache.hadoop.hive.common.classification.InterfaceAudience.Private;
+import org.apache.hadoop.hive.conf.HiveConf;
+
+@Private
+public class DefaultHiveAuthorizerFactory implements HiveAuthorizerFactory{
+  @Override
+  public HiveAuthorizer createHiveAuthorizer(HiveMetastoreClientFactory metastoreClientFactory,
+      HiveConf conf, String hiveCurrentUser) {
+    return null;
+  }
+}

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAccessController.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAccessController.java?rev=1559255&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAccessController.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAccessController.java Fri Jan 17 21:24:11 2014
@@ -0,0 +1,49 @@
+/**
+ * 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.hadoop.hive.ql.security.authorization.plugin;
+
+import java.util.List;
+
+/**
+ * Interface that is invoked by access control commands, including grant/revoke role/privileges,
+ * create/drop roles, and commands to read the state of authorization rules.
+ * Methods here have corresponding methods in HiveAuthorizer, check method documentation there.
+ */
+public interface HiveAccessController {
+
+  void grantPrivileges(List<HivePrincipal> hivePrincipals, List<HivePrivilege> hivePrivileges,
+      HivePrivilegeObject hivePrivObject, HivePrincipal grantorPrincipal, boolean grantOption);
+
+  void revokePrivileges(List<HivePrincipal> hivePrincipals, List<HivePrivilege> hivePrivileges,
+      HivePrivilegeObject hivePrivObject, HivePrincipal grantorPrincipal, boolean grantOption);
+
+  void createRole(String roleName, HivePrincipal adminGrantor);
+
+  void dropRole(String roleName);
+
+  List<String> getRoles(HivePrincipal hivePrincipal);
+
+  void grantRole(List<HivePrincipal> hivePrincipals, List<String> roles, boolean grantOption,
+      HivePrincipal grantorPrinc);
+
+  void revokeRole(List<HivePrincipal> hivePrincipals, List<String> roles, boolean grantOption,
+      HivePrincipal grantorPrinc);
+
+  List<String> getAllRoles();
+
+}

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizationValidator.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizationValidator.java?rev=1559255&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizationValidator.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizationValidator.java Fri Jan 17 21:24:11 2014
@@ -0,0 +1,42 @@
+/**
+ * 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.hadoop.hive.ql.security.authorization.plugin;
+
+import java.util.List;
+
+import org.apache.hadoop.hive.common.classification.InterfaceAudience.Public;
+import org.apache.hadoop.hive.common.classification.InterfaceStability.Evolving;
+
+/**
+ * Interface used to check if user has privileges to perform certain action.
+ * Methods here have corresponding methods in HiveAuthorizer, check method documentation there.
+ */
+@Public
+@Evolving
+public interface 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
+   */
+  void checkPrivileges(HiveOperationType hiveOpType, List<HivePrivilegeObject> inputHObjs,
+      List<HivePrivilegeObject> outputHObjs);
+
+}

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizer.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizer.java?rev=1559255&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizer.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizer.java Fri Jan 17 21:24:11 2014
@@ -0,0 +1,125 @@
+/**
+ * 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.hadoop.hive.ql.security.authorization.plugin;
+
+import java.util.List;
+
+import org.apache.hadoop.hive.common.classification.InterfaceAudience.Public;
+import org.apache.hadoop.hive.common.classification.InterfaceStability.Evolving;
+import org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider;
+
+/**
+ * Interface for hive authorization plugins.
+ * Used by the DDLTasks for access control statement,
+ * and for checking authorization from Driver.doAuthorization()
+ *
+ * This a more generic version of
+ *  {@link HiveAuthorizationProvider} that lets you define the behavior of access control
+ *  statements and does not make assumptions about the privileges needed for a hive operation.
+ * This is referred to as V2 authorizer in other parts of the code.
+ */
+@Public
+@Evolving
+public interface HiveAuthorizer {
+
+  /**
+   * Grant privileges for principals on the object
+   * @param hivePrincipals
+   * @param hivePrivileges
+   * @param hivePrivObject
+   * @param grantorPrincipal
+   * @param grantOption
+   */
+  void grantPrivileges(List<HivePrincipal> hivePrincipals, List<HivePrivilege> hivePrivileges,
+      HivePrivilegeObject hivePrivObject, HivePrincipal grantorPrincipal, boolean grantOption);
+
+  /**
+   * Revoke privileges for principals on the object
+   * @param hivePrincipals
+   * @param hivePrivileges
+   * @param hivePrivObject
+   * @param grantorPrincipal
+   * @param grantOption
+   */
+  void revokePrivileges(List<HivePrincipal> hivePrincipals, List<HivePrivilege> hivePrivileges,
+      HivePrivilegeObject hivePrivObject, HivePrincipal grantorPrincipal, boolean grantOption);
+
+  /**
+   * Create role
+   * @param roleName
+   * @param adminGrantor - The user in "[ WITH ADMIN <user> ]" clause of "create role"
+   */
+  void createRole(String roleName, HivePrincipal adminGrantor);
+
+  /**
+   * Drop role
+   * @param roleName
+   */
+  void dropRole(String roleName);
+
+  /**
+   * Get roles that this user/role belongs to
+   * @param hivePrincipal - user or role
+   * @return list of roles
+   */
+  List<String> getRoles(HivePrincipal hivePrincipal);
+
+  /**
+   * Grant roles in given roles list to principals in given hivePrincipals list
+   * @param hivePrincipals
+   * @param roles
+   * @param grantOption
+   * @param grantorPrinc
+   */
+  void grantRole(List<HivePrincipal> hivePrincipals, List<String> roles, boolean grantOption,
+      HivePrincipal grantorPrinc);
+
+  /**
+   * Revoke roles in given roles list to principals in given hivePrincipals list
+   * @param hivePrincipals
+   * @param roles
+   * @param grantOption
+   * @param grantorPrinc
+   */
+  void revokeRole(List<HivePrincipal> hivePrincipals, List<String> roles, boolean grantOption,
+      HivePrincipal grantorPrinc);
+
+  /**
+   * Check if user has privileges to do this action on these objects
+   * @param hiveOpType
+   * @param inputsHObjs
+   * @param outputHObjs
+   */
+  void checkPrivileges(HiveOperationType hiveOpType, List<HivePrivilegeObject> inputsHObjs,
+      List<HivePrivilegeObject> outputHObjs);
+
+  /**
+   * @return all existing roles
+   */
+  List<String> getAllRoles();
+
+
+  //other functions to be added -
+  //showAllRoles()
+  //showUsersInRole(rolename)
+  //showgrants(username)
+  //isSuperuser(username)
+
+
+}
+

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizerFactory.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizerFactory.java?rev=1559255&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizerFactory.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizerFactory.java Fri Jan 17 21:24:11 2014
@@ -0,0 +1,43 @@
+/**
+ * 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.hadoop.hive.ql.security.authorization.plugin;
+
+import org.apache.hadoop.hive.common.classification.InterfaceAudience.Public;
+import org.apache.hadoop.hive.common.classification.InterfaceStability.Evolving;
+import org.apache.hadoop.hive.conf.HiveConf;
+
+/**
+ * Implementation of this interface specified through hive configuration will be used to
+ * create  {@link HiveAuthorizer} instance used for hive authorization.
+ *
+ */
+@Public
+@Evolving
+public interface HiveAuthorizerFactory {
+  /**
+   * Create a new instance of HiveAuthorizer, initialized with the given objects.
+   * @param metastoreClientFactory - Use this to get the valid meta store client (IMetaStoreClient)
+   *  for the current thread. Each invocation of method in HiveAuthorizer can happen in
+   *  different thread, so get the current instance in each method invocation.
+   * @param conf - current HiveConf
+   * @param hiveCurrentUser - user for current session
+   * @return new instance of HiveAuthorizer
+   */
+  HiveAuthorizer createHiveAuthorizer(HiveMetastoreClientFactory metastoreClientFactory,
+      HiveConf conf, String hiveCurrentUser);
+}

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizerImpl.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizerImpl.java?rev=1559255&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizerImpl.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveAuthorizerImpl.java Fri Jan 17 21:24:11 2014
@@ -0,0 +1,102 @@
+/**
+ * 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.hadoop.hive.ql.security.authorization.plugin;
+
+import java.util.List;
+
+import org.apache.hadoop.hive.common.classification.InterfaceAudience.Public;
+import org.apache.hadoop.hive.common.classification.InterfaceStability.Evolving;
+
+/**
+ * Convenience implementation of HiveAuthorizer.
+ * You can customize the behavior by passing different implementations of
+ * {@link HiveAccessController} and {@link HiveAuthorizationValidator} to constructor.
+ *
+ */
+@Public
+@Evolving
+public class HiveAuthorizerImpl implements HiveAuthorizer {
+  HiveAccessController accessController;
+  HiveAuthorizationValidator authValidator;
+
+   HiveAuthorizerImpl(HiveAccessController accessController, HiveAuthorizationValidator authValidator){
+     this.accessController = accessController;
+     this.authValidator = authValidator;
+   }
+
+  @Override
+  public void grantPrivileges(List<HivePrincipal> hivePrincipals,
+      List<HivePrivilege> hivePrivileges, HivePrivilegeObject hivePrivObject,
+      HivePrincipal grantorPrincipal, boolean grantOption) {
+    accessController.grantPrivileges(hivePrincipals, hivePrivileges, hivePrivObject,
+        grantorPrincipal, grantOption);
+  }
+
+  @Override
+  public void revokePrivileges(List<HivePrincipal> hivePrincipals,
+      List<HivePrivilege> hivePrivileges, HivePrivilegeObject hivePrivObject,
+      HivePrincipal grantorPrincipal, boolean grantOption) {
+    accessController.revokePrivileges(hivePrincipals, hivePrivileges, hivePrivObject,
+        grantorPrincipal, grantOption);
+  }
+
+  @Override
+  public void createRole(String roleName, HivePrincipal adminGrantor) {
+    accessController.createRole(roleName, adminGrantor);
+  }
+
+  @Override
+  public void dropRole(String roleName) {
+    accessController.dropRole(roleName);
+  }
+
+  @Override
+  public List<String> getRoles(HivePrincipal hivePrincipal) {
+    return accessController.getRoles(hivePrincipal);
+  }
+
+  @Override
+  public void grantRole(List<HivePrincipal> hivePrincipals, List<String> roles,
+      boolean grantOption, HivePrincipal grantorPrinc) {
+    accessController.grantRole(hivePrincipals, roles, grantOption, grantorPrinc);
+  }
+
+  @Override
+  public void revokeRole(List<HivePrincipal> hivePrincipals, List<String> roles,
+      boolean grantOption, HivePrincipal grantorPrinc) {
+    accessController.revokeRole(hivePrincipals, roles, grantOption, grantorPrinc);
+  }
+
+  @Override
+  public void checkPrivileges(HiveOperationType hiveOpType, List<HivePrivilegeObject> inputHObjs,
+      List<HivePrivilegeObject> outputHObjs) {
+    authValidator.checkPrivileges(hiveOpType, inputHObjs, outputHObjs);
+  }
+
+  @Override
+  public List<String> getAllRoles() {
+    return accessController.getAllRoles();
+  }
+
+
+ // other access control functions
+
+//   void validateAuthority(HiveAction, inputs, outputs){
+//     authValidator.validateAuthority(HiveAction, inputs, outputs);
+//   }
+}

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveMetastoreClientFactory.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveMetastoreClientFactory.java?rev=1559255&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveMetastoreClientFactory.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveMetastoreClientFactory.java Fri Jan 17 21:24:11 2014
@@ -0,0 +1,30 @@
+/**
+ * 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.hadoop.hive.ql.security.authorization.plugin;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hive.common.classification.InterfaceAudience.Public;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+/**
+ * Factory for getting current valid instance of IMetaStoreClient
+ */
+@Public
+public interface HiveMetastoreClientFactory {
+  IMetaStoreClient getHiveMetastoreClient() throws IOException;
+}

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveMetastoreClientFactoryImpl.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveMetastoreClientFactoryImpl.java?rev=1559255&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveMetastoreClientFactoryImpl.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveMetastoreClientFactoryImpl.java Fri Jan 17 21:24:11 2014
@@ -0,0 +1,45 @@
+/**
+ * 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.hadoop.hive.ql.security.authorization.plugin;
+
+
+import java.io.IOException;
+
+import org.apache.hadoop.hive.common.classification.InterfaceAudience.Private;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.ql.metadata.Hive;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+/**
+ * Private implementaiton that returns instance of IMetaStoreClient
+ */
+@Private
+public class HiveMetastoreClientFactoryImpl implements HiveMetastoreClientFactory{
+
+  @Override
+  public IMetaStoreClient getHiveMetastoreClient() throws IOException {
+    try {
+      return Hive.get().getMSC();
+    } catch (MetaException e) {
+      throw new IOException(e);
+    } catch (HiveException e) {
+      throw new IOException(e);
+    }
+  }
+
+}

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java?rev=1559255&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java Fri Jan 17 21:24:11 2014
@@ -0,0 +1,110 @@
+/**
+ * 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.hadoop.hive.ql.security.authorization.plugin;
+
+import org.apache.hadoop.hive.common.classification.InterfaceAudience.Public;
+
+/**
+ * List of hive operations types.
+ */
+@Public
+public enum HiveOperationType {
+  EXPLAIN,
+  LOAD,
+  EXPORT,
+  IMPORT,
+  CREATEDATABASE,
+  DROPDATABASE,
+  SWITCHDATABASE,
+  LOCKDB,
+  UNLOCKDB,
+  DROPTABLE ,
+  DESCTABLE,
+  DESCFUNCTION,
+  MSCK,
+  ALTERTABLE_ADDCOLS,
+  ALTERTABLE_REPLACECOLS,
+  ALTERTABLE_RENAMECOL,
+  ALTERTABLE_RENAMEPART,
+  ALTERTABLE_RENAME,
+  ALTERTABLE_DROPPARTS,
+  ALTERTABLE_ADDPARTS,
+  ALTERTABLE_TOUCH,
+  ALTERTABLE_ARCHIVE,
+  ALTERTABLE_UNARCHIVE,
+  ALTERTABLE_PROPERTIES,
+  ALTERTABLE_SERIALIZER,
+  ALTERPARTITION_SERIALIZER,
+  ALTERTABLE_SERDEPROPERTIES,
+  ALTERPARTITION_SERDEPROPERTIES,
+  ALTERTABLE_CLUSTER_SORT,
+  ANALYZE_TABLE,
+  ALTERTABLE_BUCKETNUM,
+  ALTERPARTITION_BUCKETNUM,
+  SHOWDATABASES,
+  SHOWTABLES,
+  SHOWCOLUMNS,
+  SHOW_TABLESTATUS,
+  SHOW_TBLPROPERTIES,
+  SHOW_CREATETABLE,
+  SHOWFUNCTIONS,
+  SHOWINDEXES,
+  SHOWPARTITIONS,
+  SHOWLOCKS,
+  CREATEFUNCTION,
+  DROPFUNCTION,
+  CREATEMACRO,
+  DROPMACRO,
+  CREATEVIEW,
+  DROPVIEW,
+  CREATEINDEX,
+  DROPINDEX,
+  ALTERINDEX_REBUILD,
+  ALTERVIEW_PROPERTIES,
+  DROPVIEW_PROPERTIES,
+  LOCKTABLE,
+  UNLOCKTABLE,
+  CREATEROLE,
+  DROPROLE,
+  GRANT_PRIVILEGE,
+  REVOKE_PRIVILEGE,
+  SHOW_GRANT,
+  GRANT_ROLE,
+  REVOKE_ROLE,
+  SHOW_ROLES,
+  SHOW_ROLE_GRANT,
+  ALTERTABLE_PROTECTMODE,
+  ALTERPARTITION_PROTECTMODE,
+  ALTERTABLE_FILEFORMAT,
+  ALTERPARTITION_FILEFORMAT,
+  ALTERTABLE_LOCATION,
+  ALTERPARTITION_LOCATION,
+  CREATETABLE,
+  TRUNCATETABLE,
+  CREATETABLE_AS_SELECT,
+  QUERY,
+  ALTERINDEX_PROPS,
+  ALTERDATABASE,
+  DESCDATABASE,
+  ALTERTABLE_MERGEFILES,
+  ALTERPARTITION_MERGEFILES,
+  ALTERTABLE_SKEWED,
+  ALTERTBLPART_SKEWED_LOCATION,
+  ALTERVIEW_RENAME,
+
+}

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HivePrincipal.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HivePrincipal.java?rev=1559255&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HivePrincipal.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HivePrincipal.java Fri Jan 17 21:24:11 2014
@@ -0,0 +1,43 @@
+/**
+ * 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.hadoop.hive.ql.security.authorization.plugin;
+
+/**
+ * Represents the user or role in grant/revoke statements
+ */
+public class HivePrincipal {
+
+  public enum HivePrincipalType{
+    USER, ROLE, UNKNOWN
+  }
+
+  private final String name;
+  private final HivePrincipalType type;
+
+  public HivePrincipal(String name, HivePrincipalType type){
+    this.name = name;
+    this.type = type;
+  }
+  public String getName() {
+    return name;
+  }
+  public HivePrincipalType getType() {
+    return type;
+  }
+
+}

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HivePrivilege.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HivePrivilege.java?rev=1559255&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HivePrivilege.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HivePrivilege.java Fri Jan 17 21:24:11 2014
@@ -0,0 +1,42 @@
+/**
+ * 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.hadoop.hive.ql.security.authorization.plugin;
+
+import java.util.List;
+
+/**
+ * Represents the hive privilege being granted/revoked
+ */
+public class HivePrivilege {
+  private final String name;
+  private final List<String> columns;
+
+  public HivePrivilege(String name, List<String> columns){
+    this.name = name;
+    this.columns = columns;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public List<String> getColumns() {
+    return columns;
+  }
+
+}

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HivePrivilegeObject.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HivePrivilegeObject.java?rev=1559255&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HivePrivilegeObject.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HivePrivilegeObject.java Fri Jan 17 21:24:11 2014
@@ -0,0 +1,52 @@
+/**
+ * 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.hadoop.hive.ql.security.authorization.plugin;
+
+import org.apache.hadoop.hive.common.classification.InterfaceAudience.Public;
+import org.apache.hadoop.hive.common.classification.InterfaceStability.Unstable;
+
+/**
+ * Represents the object on which privilege is being granted/revoked
+ */
+@Public
+@Unstable
+public class HivePrivilegeObject {
+
+  public enum HivePrivilegeObjectType { DATABASE, TABLE, VIEW, PARTITION, URI};
+  private final HivePrivilegeObjectType type;
+  private final String dbname;
+  private final String tableviewname;
+
+  public HivePrivilegeObject(HivePrivilegeObjectType type, String dbname, String tableviewname){
+    this.type = type;
+    this.dbname = dbname;
+    this.tableviewname = tableviewname;
+  }
+
+  public HivePrivilegeObjectType getType() {
+    return type;
+  }
+
+  public String getDbname() {
+    return dbname;
+  }
+
+  public String getTableviewname() {
+    return tableviewname;
+  }
+}

Added: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveOperationType.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveOperationType.java?rev=1559255&view=auto
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveOperationType.java (added)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveOperationType.java Fri Jan 17 21:24:11 2014
@@ -0,0 +1,47 @@
+/**
+ * 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.hadoop.hive.ql.security.authorization.plugin;
+
+import static org.junit.Assert.*;
+
+import org.apache.hadoop.hive.ql.plan.HiveOperation;
+import org.junit.Test;
+
+/**
+ * Test HiveOperationType
+ */
+public class TestHiveOperationType {
+
+  /**
+   * test that all enums in {@link HiveOperation} match one in @{link HiveOperationType}
+   */
+  @Test
+  public void checkHiveOperationTypeMatch(){
+    for (HiveOperation op : HiveOperation.values()) {
+      try {
+        HiveOperationType.valueOf(op.name());
+      } catch(IllegalArgumentException ex) {
+        // if value is null or not found, exception would get thrown
+        fail("Unable to find corresponding type in HiveOperationType for " + op + " : " +  ex );
+      }
+    }
+    assertEquals("Check if HiveOperation, HiveOperationType have same number of instances",
+        HiveOperation.values().length, HiveOperationType.values().length);
+  }
+
+}