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 2015/01/30 01:05:51 UTC

svn commit: r1655891 - in /hive/branches/branch-1.0: itests/hive-unit/src/test/java/org/apache/hive/jdbc/authorization/ ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/ service/src/java/org/apache/hive/service/cli/ service/src/java/...

Author: thejas
Date: Fri Jan 30 00:05:50 2015
New Revision: 1655891

URL: http://svn.apache.org/r1655891
Log:
HIVE-9473 : sql std auth should disallow built-in udfs that allow any java methods to be called (Thejas Nair, reviewed by Jason Dere)

Added:
    hive/branches/branch-1.0/itests/hive-unit/src/test/java/org/apache/hive/jdbc/authorization/TestJdbcWithSQLAuthUDFBlacklist.java
Modified:
    hive/branches/branch-1.0/itests/hive-unit/src/test/java/org/apache/hive/jdbc/authorization/TestJdbcWithSQLAuthorization.java
    hive/branches/branch-1.0/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/SettableConfigUpdater.java
    hive/branches/branch-1.0/service/src/java/org/apache/hive/service/cli/CLIService.java
    hive/branches/branch-1.0/service/src/java/org/apache/hive/service/cli/session/SessionManager.java

Added: hive/branches/branch-1.0/itests/hive-unit/src/test/java/org/apache/hive/jdbc/authorization/TestJdbcWithSQLAuthUDFBlacklist.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-1.0/itests/hive-unit/src/test/java/org/apache/hive/jdbc/authorization/TestJdbcWithSQLAuthUDFBlacklist.java?rev=1655891&view=auto
==============================================================================
--- hive/branches/branch-1.0/itests/hive-unit/src/test/java/org/apache/hive/jdbc/authorization/TestJdbcWithSQLAuthUDFBlacklist.java (added)
+++ hive/branches/branch-1.0/itests/hive-unit/src/test/java/org/apache/hive/jdbc/authorization/TestJdbcWithSQLAuthUDFBlacklist.java Fri Jan 30 00:05:50 2015
@@ -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.hive.jdbc.authorization;
+
+import static org.junit.Assert.fail;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.HashMap;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.ql.security.SessionStateUserAuthenticator;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
+import org.apache.hive.jdbc.miniHS2.MiniHS2;
+import org.junit.After;
+import org.junit.Test;
+
+/**
+ * Test SQL standard authorization with jdbc/hiveserver2. Specifically udf blacklist behavior.
+ * This test needs to start minihs2 with specific configuration, so it is in a different
+ * test file from {@link TestJdbcWithSQLAuthorization}
+ */
+public class TestJdbcWithSQLAuthUDFBlacklist {
+  private MiniHS2 miniHS2 = null;
+
+  public void startHS2(HiveConf conf) throws Exception {
+    Class.forName(MiniHS2.getJdbcDriverName());
+
+    conf.setVar(ConfVars.HIVE_AUTHORIZATION_MANAGER, SQLStdHiveAuthorizerFactory.class.getName());
+    conf.setVar(ConfVars.HIVE_AUTHENTICATOR_MANAGER, SessionStateUserAuthenticator.class.getName());
+    conf.setBoolVar(ConfVars.HIVE_AUTHORIZATION_ENABLED, true);
+    conf.setBoolVar(ConfVars.HIVE_SUPPORT_CONCURRENCY, false);
+    conf.setBoolVar(ConfVars.HIVE_SERVER2_ENABLE_DOAS, false);
+
+    miniHS2 = new MiniHS2(conf);
+    miniHS2.start(new HashMap<String, String>());
+  }
+
+  @After
+  public void shutDownHS2() throws Exception {
+    if (miniHS2.isStarted()) {
+      miniHS2.stop();
+    }
+  }
+
+  @Test
+  public void testBlackListedUdfUsage() throws Exception {
+    HiveConf conf = new HiveConf();
+    conf.setVar(ConfVars.HIVE_SERVER2_BUILTIN_UDF_BLACKLIST, "sqrt");
+    startHS2(conf);
+
+    // create tables as user1
+    Connection hs2Conn = DriverManager.getConnection(miniHS2.getJdbcURL(), "user1", "bar");
+
+    Statement stmt = hs2Conn.createStatement();
+    String tableName1 = "test_jdbc_sql_auth_udf";
+    stmt.execute("create table " + tableName1 + "(i int) ");
+
+    verifyUDFNotAllowed(stmt, tableName1, "sqrt(1)", "sqrt");
+
+    // verify udf reflect is allowed (no exception will be thrown)
+    stmt.execute("SELECT reflect('java.lang.String', 'valueOf', 1) from " + tableName1);
+    stmt.close();
+    hs2Conn.close();
+  }
+
+  private void verifyUDFNotAllowed(Statement stmt, String tableName, String udfcall, String udfname) {
+    try {
+      stmt.execute("SELECT " + udfcall + " from " + tableName);
+      fail("Disallowed udf usage should have resulted in error");
+    } catch (SQLException e) {
+      checkAssertContains("UDF " + udfname + " is not allowed", e.getMessage());
+    }
+  }
+
+  private void checkAssertContains(String expectedSubString, String message) {
+    if (message.contains(expectedSubString)) {
+      return;
+    }
+    fail("Message [" + message + "] does not contain substring [" + expectedSubString + "]");
+  }
+
+}

Modified: hive/branches/branch-1.0/itests/hive-unit/src/test/java/org/apache/hive/jdbc/authorization/TestJdbcWithSQLAuthorization.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-1.0/itests/hive-unit/src/test/java/org/apache/hive/jdbc/authorization/TestJdbcWithSQLAuthorization.java?rev=1655891&r1=1655890&r2=1655891&view=diff
==============================================================================
--- hive/branches/branch-1.0/itests/hive-unit/src/test/java/org/apache/hive/jdbc/authorization/TestJdbcWithSQLAuthorization.java (original)
+++ hive/branches/branch-1.0/itests/hive-unit/src/test/java/org/apache/hive/jdbc/authorization/TestJdbcWithSQLAuthorization.java Fri Jan 30 00:05:50 2015
@@ -140,4 +140,39 @@ public class TestJdbcWithSQLAuthorizatio
     }
   }
 
+  @Test
+  public void testBlackListedUdfUsage() throws Exception {
+
+    // create tables as user1
+    Connection hs2Conn = getConnection("user1");
+
+    Statement stmt = hs2Conn.createStatement();
+    String tableName1 = "test_jdbc_sql_auth_udf";
+    stmt.execute("create table " + tableName1 + "(i int) ");
+
+    verifyUDFNotAllowed(stmt, tableName1, "reflect('java.lang.String', 'valueOf', 1)", "reflect");
+    verifyUDFNotAllowed(stmt, tableName1, "reflect2('java.lang.String', 'valueOf', 1)", "reflect2");
+    verifyUDFNotAllowed(stmt, tableName1, "java_method('java.lang.String', 'valueOf', 1)",
+        "java_method");
+
+    stmt.close();
+    hs2Conn.close();
+  }
+
+  private void verifyUDFNotAllowed(Statement stmt, String tableName, String udfcall, String udfname) {
+    try {
+      stmt.execute("SELECT " + udfcall + " from " + tableName);
+      fail("Disallowed udf usage should have resulted in error");
+    } catch (SQLException e) {
+      checkAssertContains("UDF " + udfname + " is not allowed", e.getMessage());
+    }
+  }
+
+  private void checkAssertContains(String expectedSubString, String message) {
+    if (message.contains(expectedSubString)) {
+      return;
+    }
+    fail("Message [" + message + "] does not contain substring [" + expectedSubString + "]");
+  }
+
 }

Modified: hive/branches/branch-1.0/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/SettableConfigUpdater.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-1.0/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/SettableConfigUpdater.java?rev=1655891&r1=1655890&r2=1655891&view=diff
==============================================================================
--- hive/branches/branch-1.0/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/SettableConfigUpdater.java (original)
+++ hive/branches/branch-1.0/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/SettableConfigUpdater.java Fri Jan 30 00:05:50 2015
@@ -58,6 +58,13 @@ public class SettableConfigUpdater {
     }
 
     hiveConf.setModifiableWhiteListRegex(whiteListParamsStr);
+
+    // disallow udfs that can potentially allow untrusted code execution
+    // if admin has already customized this list, honor that
+    String curBlackList = hiveConf.getVar(ConfVars.HIVE_SERVER2_BUILTIN_UDF_BLACKLIST);
+    if (curBlackList == null || curBlackList.trim().isEmpty()) {
+      hiveConf.setVar(ConfVars.HIVE_SERVER2_BUILTIN_UDF_BLACKLIST, "reflect,reflect2,java_method");
+    }
   }
 
 }

Modified: hive/branches/branch-1.0/service/src/java/org/apache/hive/service/cli/CLIService.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-1.0/service/src/java/org/apache/hive/service/cli/CLIService.java?rev=1655891&r1=1655890&r2=1655891&view=diff
==============================================================================
--- hive/branches/branch-1.0/service/src/java/org/apache/hive/service/cli/CLIService.java (original)
+++ hive/branches/branch-1.0/service/src/java/org/apache/hive/service/cli/CLIService.java Fri Jan 30 00:05:50 2015
@@ -43,13 +43,11 @@ import org.apache.hive.service.Composite
 import org.apache.hive.service.ServiceException;
 import org.apache.hive.service.auth.HiveAuthFactory;
 import org.apache.hive.service.cli.operation.Operation;
+import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hive.service.cli.session.SessionManager;
 import org.apache.hive.service.cli.thrift.TProtocolVersion;
 import org.apache.hive.service.server.HiveServer2;
 
-import com.google.common.base.Splitter;
-import com.google.common.collect.Lists;
-
 /**
  * CLIService.
  *
@@ -79,6 +77,11 @@ public class CLIService extends Composit
 
   @Override
   public synchronized void init(HiveConf hiveConf) {
+    try {
+      applyAuthorizationConfigPolicy(hiveConf);
+    } catch (HiveException e) {
+      throw new RuntimeException("Error applying authorization policy on hive configuration", e);
+    }
     this.hiveConf = hiveConf;
     sessionManager = new SessionManager(hiveServer2);
     addService(sessionManager);
@@ -112,6 +115,15 @@ public class CLIService extends Composit
     super.init(hiveConf);
   }
 
+  private void applyAuthorizationConfigPolicy(HiveConf newHiveConf) throws HiveException {
+    // authorization setup using SessionState should be revisited eventually, as
+    // authorization and authentication are not session specific settings
+    SessionState ss = new SessionState(newHiveConf);
+    ss.setIsHiveServerQuery(true);
+    SessionState.start(ss);
+    ss.applyAuthorizationPolicy();
+  }
+
   private void setupBlockedUdfs() {
     FunctionRegistry.setupPermissionsForBuiltinUDFs(
         hiveConf.getVar(ConfVars.HIVE_SERVER2_BUILTIN_UDF_WHITELIST),

Modified: hive/branches/branch-1.0/service/src/java/org/apache/hive/service/cli/session/SessionManager.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-1.0/service/src/java/org/apache/hive/service/cli/session/SessionManager.java?rev=1655891&r1=1655890&r2=1655891&view=diff
==============================================================================
--- hive/branches/branch-1.0/service/src/java/org/apache/hive/service/cli/session/SessionManager.java (original)
+++ hive/branches/branch-1.0/service/src/java/org/apache/hive/service/cli/session/SessionManager.java Fri Jan 30 00:05:50 2015
@@ -36,8 +36,6 @@ import org.apache.commons.logging.LogFac
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.ql.hooks.HookUtils;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hive.service.CompositeService;
 import org.apache.hive.service.cli.HiveSQLException;
 import org.apache.hive.service.cli.SessionHandle;
@@ -76,11 +74,6 @@ public class SessionManager extends Comp
 
   @Override
   public synchronized void init(HiveConf hiveConf) {
-    try {
-      applyAuthorizationConfigPolicy(hiveConf);
-    } catch (HiveException e) {
-      throw new RuntimeException("Error applying authorization policy on hive configuration", e);
-    }
     this.hiveConf = hiveConf;
     //Create operation log root directory, if operation logging is enabled
     if (hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_LOGGING_OPERATION_ENABLED)) {
@@ -116,15 +109,6 @@ public class SessionManager extends Comp
         hiveConf, ConfVars.HIVE_SERVER2_IDLE_SESSION_TIMEOUT, TimeUnit.MILLISECONDS);
   }
 
-  private void applyAuthorizationConfigPolicy(HiveConf newHiveConf) throws HiveException {
-    // authorization setup using SessionState should be revisited eventually, as
-    // authorization and authentication are not session specific settings
-    SessionState ss = new SessionState(newHiveConf);
-    ss.setIsHiveServerQuery(true);
-    SessionState.start(ss);
-    ss.applyAuthorizationPolicy();
-  }
-
   private void initOperationLogRootDir() {
     operationLogRootDir = new File(
         hiveConf.getVar(ConfVars.HIVE_SERVER2_LOGGING_OPERATION_LOG_LOCATION));