You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by ad...@apache.org on 2015/09/09 01:25:24 UTC

[4/5] drill git commit: DRILL-3622: When user authentication is enabled, enforce admin privileges to update SYSTEM options

DRILL-3622: When user authentication is enabled, enforce admin privileges to update SYSTEM options

+ define what user is considered an admin
+ remove a stray file in test module (exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/RootResource.java)


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/41fc9ca5
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/41fc9ca5
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/41fc9ca5

Branch: refs/heads/master
Commit: 41fc9ca52c6983e78b2a609b96b087814a3c7969
Parents: 1601a7c
Author: vkorukanti <ve...@gmail.com>
Authored: Tue Sep 8 17:01:14 2015 +0000
Committer: adeneche <ad...@gmail.com>
Committed: Tue Sep 8 15:35:45 2015 -0700

----------------------------------------------------------------------
 .../org/apache/drill/exec/ExecConstants.java    |  16 +++
 .../org/apache/drill/exec/ops/QueryContext.java |   4 +
 .../planner/sql/handlers/SetOptionHandler.java  |  21 +++-
 .../user/security/UserAuthenticatorFactory.java |   4 +-
 .../server/options/SystemOptionManager.java     |   2 +
 .../exec/server/options/TypeValidators.java     |  16 +++
 .../drill/exec/util/ImpersonationUtil.java      |  40 +++++++
 .../java/org/apache/drill/BaseTestQuery.java    |  15 ++-
 .../security/UserAuthenticatorTestImpl.java     |  19 +++-
 .../exec/server/TestOptionsAuthEnabled.java     | 112 +++++++++++++++++++
 .../drill/exec/server/rest/RootResource.java    |  26 -----
 11 files changed, 243 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/41fc9ca5/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
index 140e9a8..0f6a5bb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java
@@ -19,6 +19,7 @@ package org.apache.drill.exec;
 
 import org.apache.drill.exec.physical.impl.common.HashTable;
 import org.apache.drill.exec.server.options.OptionValidator;
+import org.apache.drill.exec.server.options.TypeValidators.AdminOptionValidator;
 import org.apache.drill.exec.server.options.TypeValidators.BooleanValidator;
 import org.apache.drill.exec.server.options.TypeValidators.DoubleValidator;
 import org.apache.drill.exec.server.options.TypeValidators.EnumeratedStringValidator;
@@ -29,6 +30,7 @@ import org.apache.drill.exec.server.options.TypeValidators.RangeLongValidator;
 import org.apache.drill.exec.server.options.TypeValidators.RangeDoubleValidator;
 import org.apache.drill.exec.server.options.TypeValidators.StringValidator;
 import org.apache.drill.exec.testing.ExecutionControls;
+import org.apache.drill.exec.util.ImpersonationUtil;
 
 public interface ExecConstants {
   public static final String ZK_RETRY_TIMES = "drill.exec.zk.retry.count";
@@ -254,4 +256,18 @@ public interface ExecConstants {
 
   public static final String CTAS_PARTITIONING_HASH_DISTRIBUTE = "store.partition.hash_distribute";
   public static final BooleanValidator CTAS_PARTITIONING_HASH_DISTRIBUTE_VALIDATOR = new BooleanValidator(CTAS_PARTITIONING_HASH_DISTRIBUTE, false);
+
+  /**
+   * Option whose value is a comma separated list of admin usernames. Admin users are users who have special privileges
+   * such as changing system options.
+   */
+  String ADMIN_USERS_KEY = "security.admin.users";
+  StringValidator ADMIN_USERS_VALIDATOR =
+      new AdminOptionValidator(ADMIN_USERS_KEY, ImpersonationUtil.getProcessUserName());
+
+  /**
+   * Option whose value is a comma separated list of admin usergroups.
+   */
+  String ADMIN_USER_GROUPS_KEY = "security.admin.user_groups";
+  StringValidator ADMIN_USER_GROUPS_VALIDATOR = new AdminOptionValidator(ADMIN_USER_GROUPS_KEY, "");
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/41fc9ca5/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
index 1cd67ac..238907d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
@@ -217,6 +217,10 @@ public class QueryContext implements AutoCloseable, OptimizerRulesContext {
      return getConfig().getBoolean(ExecConstants.IMPERSONATION_ENABLED);
   }
 
+  public boolean isUserAuthenticationEnabled() {
+    return getConfig().getBoolean(ExecConstants.USER_AUTHENTICATION_ENABLED);
+  }
+
   public DrillOperatorTable getDrillOperatorTable() {
     return table;
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/41fc9ca5/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SetOptionHandler.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SetOptionHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SetOptionHandler.java
index 2b1a230..85ab528 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SetOptionHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SetOptionHandler.java
@@ -26,17 +26,21 @@ import org.apache.calcite.tools.ValidationException;
 
 import org.apache.calcite.util.NlsString;
 import org.apache.drill.common.exceptions.ExpressionParsingException;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.ops.QueryContext;
 import org.apache.drill.exec.physical.PhysicalPlan;
 import org.apache.drill.exec.planner.sql.DirectPlan;
 import org.apache.drill.exec.server.options.OptionValue;
+import org.apache.drill.exec.server.options.OptionValue.OptionType;
+import org.apache.drill.exec.util.ImpersonationUtil;
 import org.apache.drill.exec.work.foreman.ForemanSetupException;
 import org.apache.calcite.sql.SqlLiteral;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlSetOption;
 
 public class SetOptionHandler extends AbstractSqlHandler {
-//  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SetOptionHandler.class);
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SetOptionHandler.class);
 
   private final QueryContext context;
 
@@ -65,6 +69,21 @@ public class SetOptionHandler extends AbstractSqlHandler {
         default:
           throw new ValidationException("Invalid OPTION scope. Scope must be SESSION or SYSTEM.");
       }
+
+      if (type == OptionType.SYSTEM) {
+        // If the user authentication is enabled, make sure the user who is trying to change the system option has
+        // administrative privileges.
+        if (context.isUserAuthenticationEnabled() &&
+            !ImpersonationUtil.hasAdminPrivileges(
+                context.getQueryUserName(),
+                context.getOptions().getOption(ExecConstants.ADMIN_USERS_KEY).string_val,
+                context.getOptions().getOption(ExecConstants.ADMIN_USER_GROUPS_KEY).string_val)) {
+          throw UserException.permissionError()
+              .message("Not authorized to change SYSTEM options.")
+              .build(logger);
+        }
+      }
+
       final OptionValue optionValue = createOptionValue(name, type, (SqlLiteral) value);
       context.getOptions().setOption(optionValue);
     }else{

http://git-wip-us.apache.org/repos/asf/drill/blob/41fc9ca5/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/security/UserAuthenticatorFactory.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/security/UserAuthenticatorFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/security/UserAuthenticatorFactory.java
index 51a5979..0fe302d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/security/UserAuthenticatorFactory.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/security/UserAuthenticatorFactory.java
@@ -92,13 +92,13 @@ public class UserAuthenticatorFactory {
           return authenticator;
         } catch(IllegalArgumentException | IllegalAccessException | InstantiationException e) {
           throw new DrillbitStartupException(
-              String.format("Failed to create and initialize the UserAuthenticator class '{}'",
+              String.format("Failed to create and initialize the UserAuthenticator class '%s'",
                   clazz.getCanonicalName()), e);
         }
       }
     }
 
-    String errMsg = String.format("Failed to find the implementation of '{}' for type '{}'",
+    String errMsg = String.format("Failed to find the implementation of '%s' for type '%s'",
         UserAuthenticator.class.getCanonicalName(), authImplConfigured);
     logger.error(errMsg);
     throw new DrillbitStartupException(errMsg);

http://git-wip-us.apache.org/repos/asf/drill/blob/41fc9ca5/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
index 1b9906d..118f7ad 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java
@@ -111,6 +111,8 @@ public class SystemOptionManager extends BaseOptionManager {
       ExecConstants.NEW_VIEW_DEFAULT_PERMS_VALIDATOR,
       ExecConstants.USE_OLD_ASSIGNMENT_CREATOR_VALIDATOR,
       ExecConstants.CTAS_PARTITIONING_HASH_DISTRIBUTE_VALIDATOR,
+      ExecConstants.ADMIN_USERS_VALIDATOR,
+      ExecConstants.ADMIN_USER_GROUPS_VALIDATOR,
       QueryClassLoader.JAVA_COMPILER_VALIDATOR,
       QueryClassLoader.JAVA_COMPILER_JANINO_MAXSIZE,
       QueryClassLoader.JAVA_COMPILER_DEBUG,

http://git-wip-us.apache.org/repos/asf/drill/blob/41fc9ca5/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/TypeValidators.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/TypeValidators.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/TypeValidators.java
index 73f067b..53cd4f3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/TypeValidators.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/options/TypeValidators.java
@@ -133,6 +133,22 @@ public class TypeValidators {
     }
   }
 
+  public static class AdminOptionValidator extends StringValidator {
+    public AdminOptionValidator(String name, String def) {
+      super(name, def);
+    }
+
+    @Override
+    public void validate(OptionValue v) {
+      if (v.type != OptionType.SYSTEM) {
+        throw UserException.validationError()
+            .message("Admin related settings can only be set at SYSTEM level scope. Given scope '%s'.", v.type)
+            .build(logger);
+      }
+      super.validate(v);
+    }
+  }
+
   /**
    * Validator that checks if the given value is included in a list of acceptable values. Case insensitive.
    */

http://git-wip-us.apache.org/repos/asf/drill/blob/41fc9ca5/exec/java-exec/src/main/java/org/apache/drill/exec/util/ImpersonationUtil.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/util/ImpersonationUtil.java b/exec/java-exec/src/main/java/org/apache/drill/exec/util/ImpersonationUtil.java
index aa766be..37fdc40 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/util/ImpersonationUtil.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/util/ImpersonationUtil.java
@@ -17,7 +17,9 @@
  */
 package org.apache.drill.exec.util;
 
+import com.google.common.base.Splitter;
 import com.google.common.base.Strings;
+import com.google.common.collect.Sets;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.exec.ops.OperatorStats;
 import org.apache.drill.exec.store.dfs.DrillFileSystem;
@@ -26,6 +28,7 @@ import org.apache.hadoop.security.UserGroupInformation;
 
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
+import java.util.Set;
 
 /**
  * Utilities for impersonation purpose.
@@ -33,6 +36,8 @@ import java.security.PrivilegedExceptionAction;
 public class ImpersonationUtil {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ImpersonationUtil.class);
 
+  private static final Splitter SPLITTER = Splitter.on(',').trimResults().omitEmptyStrings();
+
   /**
    * Create and return proxy user {@link org.apache.hadoop.security.UserGroupInformation} of operator owner if operator
    * owner is valid. Otherwise create and return proxy user {@link org.apache.hadoop.security.UserGroupInformation} for
@@ -152,4 +157,39 @@ public class ImpersonationUtil {
 
     return fs;
   }
+
+  /**
+   * Given admin user/group list, finds whether the given username has admin privileges.
+   *
+   * @param userName User who is checked for administrative privileges.
+   * @param adminUsers Comma separated list of admin usernames,
+   * @param adminGroups Comma separated list of admin usergroups
+   * @return
+   */
+  public static boolean hasAdminPrivileges(final String userName, final String adminUsers, final String adminGroups) {
+    // Process user is by default an admin
+    if (getProcessUserName().equals(userName)) {
+      return true;
+    }
+
+    final Set<String> adminUsersSet = Sets.newHashSet(SPLITTER.split(adminUsers));
+    if (adminUsersSet.contains(userName)) {
+      return true;
+    }
+
+    final UserGroupInformation ugi = createProxyUgi(userName);
+    final String[] userGroups = ugi.getGroupNames();
+    if (userGroups == null || userGroups.length == 0) {
+      return false;
+    }
+
+    final Set<String> adminUserGroupsSet = Sets.newHashSet(SPLITTER.split(adminGroups));
+    for (String userGroup : userGroups) {
+      if (adminUserGroupsSet.contains(userGroup)) {
+        return true;
+      }
+    }
+
+    return false;
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/41fc9ca5/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java b/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java
index eaf8765..1381949 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java
@@ -44,6 +44,7 @@ import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.rpc.user.ConnectionThrottle;
 import org.apache.drill.exec.rpc.user.QueryDataBatch;
 import org.apache.drill.exec.rpc.user.UserResultsListener;
+import org.apache.drill.exec.rpc.user.UserSession;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.server.RemoteServiceSet;
@@ -211,8 +212,20 @@ public class BaseTestQuery extends ExecTest {
    * @param user
    */
   public static void updateClient(String user) throws Exception {
+    updateClient(user, null);
+  }
+
+  /*
+   * Close the current <i>client</i> and open a new client for the given user and password credentials. Tests
+   * executed after this method call use the new <i>client</i>.
+   * @param user
+   */
+  public static void updateClient(final String user, final String password) throws Exception {
     final Properties props = new Properties();
-    props.setProperty("user", user);
+    props.setProperty(UserSession.USER, user);
+    if (password != null) {
+      props.setProperty(UserSession.PASSWORD, password);
+    }
     updateClient(props);
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/41fc9ca5/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/UserAuthenticatorTestImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/UserAuthenticatorTestImpl.java b/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/UserAuthenticatorTestImpl.java
index c89471f..dc30797 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/UserAuthenticatorTestImpl.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/UserAuthenticatorTestImpl.java
@@ -17,9 +17,10 @@
  */
 package org.apache.drill.exec.rpc.user.security;
 
-import com.google.common.base.Strings;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.exec.exception.DrillbitStartupException;
+import org.apache.drill.exec.util.ImpersonationUtil;
+import org.apache.hadoop.security.UserGroupInformation;
 
 import java.io.IOException;
 
@@ -33,8 +34,20 @@ public class UserAuthenticatorTestImpl implements UserAuthenticator {
 
   public static final String TEST_USER_1 = "testUser1";
   public static final String TEST_USER_2 = "testUser2";
+  public static final String ADMIN_USER = "admin";
+  public static final String PROCESS_USER = ImpersonationUtil.getProcessUserName();
   public static final String TEST_USER_1_PASSWORD = "testUser1Password";
   public static final String TEST_USER_2_PASSWORD = "testUser2Password";
+  public static final String ADMIN_USER_PASSWORD = "adminUserPw";
+  public static final String PROCESS_USER_PASSWORD = "processUserPw";
+
+  public static final String ADMIN_GROUP = "admingrp";
+
+  static {
+    UserGroupInformation.createUserForTesting("testUser1", new String[]{"g1", ADMIN_GROUP});
+    UserGroupInformation.createUserForTesting("testUser2", new String[]{ "g1" });
+    UserGroupInformation.createUserForTesting("admin", new String[]{ ADMIN_GROUP });
+  }
 
   @Override
   public void setup(DrillConfig drillConfig) throws DrillbitStartupException {
@@ -50,7 +63,9 @@ public class UserAuthenticatorTestImpl implements UserAuthenticator {
     }
 
     if (!(TEST_USER_1.equals(user) && TEST_USER_1_PASSWORD.equals(password)) &&
-        !(TEST_USER_2.equals(user) && TEST_USER_2_PASSWORD.equals(password))) {
+        !(TEST_USER_2.equals(user) && TEST_USER_2_PASSWORD.equals(password)) &&
+        !(ADMIN_USER.equals(user) && ADMIN_USER_PASSWORD.equals(password)) &&
+        !(PROCESS_USER.equals(user) && PROCESS_USER_PASSWORD.equals(password))) {
       throw new UserAuthenticationException();
     }
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/41fc9ca5/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestOptionsAuthEnabled.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestOptionsAuthEnabled.java b/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestOptionsAuthEnabled.java
new file mode 100644
index 0000000..b029caa
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/server/TestOptionsAuthEnabled.java
@@ -0,0 +1,112 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.server;
+
+import org.apache.drill.BaseTestQuery;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.rpc.user.security.UserAuthenticatorTestImpl;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.Properties;
+
+import static org.apache.drill.exec.rpc.user.security.UserAuthenticatorTestImpl.ADMIN_GROUP;
+import static org.apache.drill.exec.rpc.user.security.UserAuthenticatorTestImpl.ADMIN_USER;
+import static org.apache.drill.exec.rpc.user.security.UserAuthenticatorTestImpl.ADMIN_USER_PASSWORD;
+import static org.apache.drill.exec.rpc.user.security.UserAuthenticatorTestImpl.PROCESS_USER;
+import static org.apache.drill.exec.rpc.user.security.UserAuthenticatorTestImpl.PROCESS_USER_PASSWORD;
+import static org.apache.drill.exec.rpc.user.security.UserAuthenticatorTestImpl.TEST_USER_1;
+import static org.apache.drill.exec.rpc.user.security.UserAuthenticatorTestImpl.TEST_USER_1_PASSWORD;
+import static org.apache.drill.exec.rpc.user.security.UserAuthenticatorTestImpl.TEST_USER_2;
+import static org.apache.drill.exec.rpc.user.security.UserAuthenticatorTestImpl.TEST_USER_2_PASSWORD;
+
+/**
+ * Test setting system scoped options with user authentication enabled. (DRILL-3622)
+ */
+public class TestOptionsAuthEnabled extends BaseTestQuery {
+  private static final String setSysOptionQuery =
+      String.format("ALTER SYSTEM SET `%s` = %d;", ExecConstants.SLICE_TARGET, 200);
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    // Create a new DrillConfig which has user authentication enabled and test authenticator set
+    final Properties props = cloneDefaultTestConfigProperties();
+    props.setProperty(ExecConstants.USER_AUTHENTICATION_ENABLED, "true");
+    props.setProperty(ExecConstants.USER_AUTHENTICATOR_IMPL, UserAuthenticatorTestImpl.TYPE);
+
+    updateTestCluster(1, DrillConfig.create(props));
+
+    updateClient(PROCESS_USER, PROCESS_USER_PASSWORD);
+
+    // Add user "admin" to admin username list
+    test(String.format("ALTER SYSTEM SET `%s`='%s,%s'", ExecConstants.ADMIN_USERS_KEY, ADMIN_USER, PROCESS_USER));
+
+    // Set "admingrp" to admin username list
+    test(String.format("ALTER SYSTEM SET `%s`='%s'", ExecConstants.ADMIN_USER_GROUPS_KEY, ADMIN_GROUP));
+  }
+
+  @Test
+  public void updateSysOptAsAdminUser() throws Exception {
+    updateClient(ADMIN_USER, ADMIN_USER_PASSWORD);
+    setOptHelper();
+  }
+
+  @Test
+  public void updateSysOptAsNonAdminUser() throws Exception {
+    updateClient(TEST_USER_2, TEST_USER_2_PASSWORD);
+    errorMsgTestHelper(setSysOptionQuery, "Not authorized to change SYSTEM options.");
+  }
+
+  @Test
+  public void updateSysOptAsUserInAdminGroup() throws Exception {
+    updateClient(TEST_USER_1, TEST_USER_1_PASSWORD);
+    setOptHelper();
+  }
+
+  @Test
+  public void trySettingAdminOptsAtSessionScopeAsAdmin() throws Exception {
+    updateClient(ADMIN_USER, ADMIN_USER_PASSWORD);
+    final String setOptionQuery =
+        String.format("ALTER SESSION SET `%s`='%s,%s'", ExecConstants.ADMIN_USERS_KEY, ADMIN_USER, PROCESS_USER);
+    errorMsgTestHelper(setOptionQuery, "Admin related settings can only be set at SYSTEM level scope");
+  }
+
+  @Test
+  public void trySettingAdminOptsAtSessionScopeAsNonAdmin() throws Exception {
+    updateClient(TEST_USER_2, TEST_USER_2_PASSWORD);
+    final String setOptionQuery =
+        String.format("ALTER SESSION SET `%s`='%s,%s'", ExecConstants.ADMIN_USERS_KEY, ADMIN_USER, PROCESS_USER);
+    errorMsgTestHelper(setOptionQuery, "Admin related settings can only be set at SYSTEM level scope");
+  }
+
+  private void setOptHelper() throws Exception {
+    try {
+      test(setSysOptionQuery);
+      testBuilder()
+          .sqlQuery(String.format("SELECT num_val FROM sys.options WHERE name = '%s' AND type = 'SYSTEM'",
+              ExecConstants.SLICE_TARGET))
+          .unOrdered()
+          .baselineColumns("num_val")
+          .baselineValues(200L)
+          .go();
+    } finally {
+      test(String.format("ALTER SYSTEM SET `%s` = %d;", ExecConstants.SLICE_TARGET, ExecConstants.SLICE_TARGET_DEFAULT));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/41fc9ca5/exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/RootResource.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/RootResource.java b/exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/RootResource.java
deleted file mode 100644
index 59adad9..0000000
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/server/rest/RootResource.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.server.rest;
-
-import javax.ws.rs.Path;
-
-@Path("/")
-public class RootResource {
-  public int hi = 5;
-  public String blue = "yo";
-}