aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorSudheesh Katkam <skatkam@maprtech.com>2016-03-05 15:18:28 -0800
committerSudheesh Katkam <skatkam@maprtech.com>2016-03-05 15:19:20 -0800
commitb67c553833b0d49af4489655128d6e80e1e7be46 (patch)
treec95b1d28b5e335f7ab44b3664fddaf9f38b73c60
parentde2d062a6a5f4077bfaa254e170ecdbb7fa0dc8d (diff)
DRILL-4281: Support authorized proxy users to impersonate other users
closes #400
-rw-r--r--contrib/native/client/src/clientlib/drillClient.cpp1
-rw-r--r--contrib/native/client/src/include/drill/common.hpp1
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java18
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/InboundImpersonationManager.java191
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java10
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserSession.java43
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/server/options/SystemOptionManager.java1
-rw-r--r--exec/java-exec/src/main/java/org/apache/drill/exec/util/ImpersonationUtil.java7
-rw-r--r--exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/BaseTestImpersonation.java3
-rw-r--r--exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/TestInboundImpersonation.java165
-rw-r--r--exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/TestInboundImpersonationPrivileges.java144
-rw-r--r--exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/testing/UserAuthenticatorTestImpl.java29
-rw-r--r--exec/java-exec/src/test/resources/inbound_impersonation_policies.json14
13 files changed, 609 insertions, 18 deletions
diff --git a/contrib/native/client/src/clientlib/drillClient.cpp b/contrib/native/client/src/clientlib/drillClient.cpp
index 812483d4c..708793867 100644
--- a/contrib/native/client/src/clientlib/drillClient.cpp
+++ b/contrib/native/client/src/clientlib/drillClient.cpp
@@ -140,6 +140,7 @@ const std::map<std::string, uint32_t> DrillUserProperties::USER_PROPERTIES=boos
( USERPROP_USERNAME, USERPROP_FLAGS_SERVERPROP|USERPROP_FLAGS_USERNAME|USERPROP_FLAGS_STRING )
( USERPROP_PASSWORD, USERPROP_FLAGS_SERVERPROP|USERPROP_FLAGS_PASSWORD)
( USERPROP_SCHEMA, USERPROP_FLAGS_SERVERPROP|USERPROP_FLAGS_STRING)
+ ( USERPROP_IMPERSONATION_TARGET, USERPROP_FLAGS_SERVERPROP|USERPROP_FLAGS_STRING)
( USERPROP_USESSL, USERPROP_FLAGS_BOOLEAN|USERPROP_FLAGS_SSLPROP)
( USERPROP_FILEPATH, USERPROP_FLAGS_STRING|USERPROP_FLAGS_SSLPROP|USERPROP_FLAGS_FILEPATH)
( USERPROP_FILENAME, USERPROP_FLAGS_STRING|USERPROP_FLAGS_SSLPROP|USERPROP_FLAGS_FILENAME)
diff --git a/contrib/native/client/src/include/drill/common.hpp b/contrib/native/client/src/include/drill/common.hpp
index da411499d..bb8e2b4a3 100644
--- a/contrib/native/client/src/include/drill/common.hpp
+++ b/contrib/native/client/src/include/drill/common.hpp
@@ -140,6 +140,7 @@ typedef enum{
#define USERPROP_USESSL "useSSL" // Not implemented yet
#define USERPROP_FILEPATH "pemLocation" // Not implemented yet
#define USERPROP_FILENAME "pemFile" // Not implemented yet
+#define USERPROP_IMPERSONATION_TARGET "impersonation_target"
// Bitflags to describe user properties
// Used in DrillUserProperties::USER_PROPERTIES
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 179924e47..b8f25ad31 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
@@ -18,6 +18,7 @@
package org.apache.drill.exec;
import org.apache.drill.exec.physical.impl.common.HashTable;
+import org.apache.drill.exec.rpc.user.InboundImpersonationManager;
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;
@@ -282,4 +283,21 @@ public interface ExecConstants {
*/
String ADMIN_USER_GROUPS_KEY = "security.admin.user_groups";
StringValidator ADMIN_USER_GROUPS_VALIDATOR = new AdminOptionValidator(ADMIN_USER_GROUPS_KEY, "");
+
+ /**
+ * Option whose value is a string representing list of inbound impersonation policies.
+ *
+ * Impersonation policy format:
+ * [
+ * {
+ * proxy_principals : { users : [“...”], groups : [“...”] },
+ * target_principals : { users : [“...”], groups : [“...”] }
+ * },
+ * ...
+ * ]
+ */
+ String IMPERSONATION_POLICIES_KEY = "exec.impersonation.inbound_policies";
+ StringValidator IMPERSONATION_POLICY_VALIDATOR =
+ new InboundImpersonationManager.InboundImpersonationPolicyValidator(IMPERSONATION_POLICIES_KEY, "[]");
+
}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/InboundImpersonationManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/InboundImpersonationManager.java
new file mode 100644
index 000000000..06c3fe275
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/InboundImpersonationManager.java
@@ -0,0 +1,191 @@
+/**
+ * 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.rpc.user;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Sets;
+import org.apache.drill.common.exceptions.DrillRuntimeException;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.proto.UserBitShared.UserCredentials;
+import org.apache.drill.exec.server.options.OptionValue;
+import org.apache.drill.exec.server.options.TypeValidators;
+import org.apache.drill.exec.util.ImpersonationUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Helper class to manage inbound impersonation.
+ * <p/>
+ * Impersonation policies format:
+ * [
+ * {
+ * proxy_principals : { users : [“...”], groups : [“...”] },
+ * target_principals : { users : [“...”], groups : [“...”] }
+ * },
+ * {
+ * proxy_principals : { users : [“...”], groups : [“...”] },
+ * target_principals : { users : [“...”], groups : [“...”] }
+ * },
+ * ...
+ * ]
+ */
+public class InboundImpersonationManager {
+ private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(InboundImpersonationManager.class);
+
+ private static final String STAR = "*";
+
+ private static final ObjectMapper impersonationPolicyMapper = new ObjectMapper();
+
+ static {
+ impersonationPolicyMapper.configure(JsonGenerator.Feature.QUOTE_FIELD_NAMES, false);
+ impersonationPolicyMapper.configure(JsonParser.Feature.ALLOW_UNQUOTED_FIELD_NAMES, true);
+ }
+
+ private static class ImpersonationPolicy {
+ public UserGroupDefinition proxy_principals = new UserGroupDefinition();
+ public UserGroupDefinition target_principals = new UserGroupDefinition();
+ }
+
+ private static class UserGroupDefinition {
+ public Set<String> users = Sets.newHashSet();
+ public Set<String> groups = Sets.newHashSet();
+ }
+
+ private static List<ImpersonationPolicy> deserializeImpersonationPolicies(final String impersonationPolicies)
+ throws IOException {
+ return impersonationPolicyMapper.readValue(impersonationPolicies,
+ new TypeReference<List<ImpersonationPolicy>>() {});
+ }
+
+ /**
+ * Validator for impersonation policies.
+ */
+ public static class InboundImpersonationPolicyValidator extends TypeValidators.AdminOptionValidator {
+
+ public InboundImpersonationPolicyValidator(String name, String def) {
+ super(name, def);
+ }
+
+ @Override
+ public void validate(OptionValue v) {
+ super.validate(v);
+
+ final List<ImpersonationPolicy> policies;
+ try {
+ policies = deserializeImpersonationPolicies(v.string_val);
+ } catch (final IOException e) {
+ throw UserException.validationError()
+ .message("Invalid impersonation policies.\nDetails: %s", e.getMessage())
+ .build(logger);
+ }
+
+ for (final ImpersonationPolicy policy : policies) {
+ if (policy.proxy_principals.users.contains(STAR) ||
+ policy.proxy_principals.groups.contains(STAR)) {
+ throw UserException.validationError()
+ .message("Proxy principals cannot have a wildcard entry.")
+ .build(logger);
+ }
+ }
+ }
+ }
+
+ /**
+ * Checks if the proxy user is authorized to impersonate the target user based on the policies.
+ *
+ * @param proxyName proxy user name
+ * @param targetName target user name
+ * @param policies impersonation policies
+ * @return true iff proxy user is authorized to impersonate the target user
+ */
+ private static boolean hasImpersonationPrivileges(final String proxyName, final String targetName,
+ final List<ImpersonationPolicy> policies) {
+ final UserGroupInformation proxyUgi = ImpersonationUtil.createProxyUgi(proxyName);
+ final Set<String> proxyGroups = Sets.newHashSet(proxyUgi.getGroupNames());
+ final UserGroupInformation targetUgi = ImpersonationUtil.createProxyUgi(targetName);
+ final Set<String> targetGroups = Sets.newHashSet(targetUgi.getGroupNames());
+ for (final ImpersonationPolicy definition : policies) {
+ // check if proxy user qualifies within this policy
+ if (definition.proxy_principals.users.contains(proxyName) ||
+ !Sets.intersection(definition.proxy_principals.groups, proxyGroups).isEmpty()) {
+ // check if target qualifies within this policy
+ if (definition.target_principals.users.contains(targetName) ||
+ definition.target_principals.users.contains(STAR) ||
+ !Sets.intersection(definition.target_principals.groups, targetGroups).isEmpty() ||
+ definition.target_principals.groups.contains(STAR)) {
+ return true;
+ }
+ }
+ }
+ return false;
+ }
+
+ @VisibleForTesting
+ public static boolean hasImpersonationPrivileges(final String proxyName, final String targetName,
+ final String policiesString) throws IOException {
+ return hasImpersonationPrivileges(proxyName, targetName,
+ deserializeImpersonationPolicies(policiesString));
+ }
+
+
+ private List<ImpersonationPolicy> impersonationPolicies;
+ private String policiesString; // used to test if policies changed
+
+ /**
+ * Check if the current session user, as a proxy user, is authorized to impersonate the given target user
+ * based on the system's impersonation policies.
+ *
+ * @param targetName target user name
+ * @param session user session
+ */
+ public void replaceUserOnSession(final String targetName, final UserSession session) {
+ final String policiesString = session.getOptions()
+ .getOption(ExecConstants.IMPERSONATION_POLICY_VALIDATOR);
+ if (!policiesString.equals(this.policiesString)) {
+ try {
+ impersonationPolicies = deserializeImpersonationPolicies(policiesString);
+ this.policiesString = policiesString;
+ } catch (final IOException e) {
+ // This never happens. Impersonation policies must have been validated.
+ logger.warn("Impersonation policies must have been validated.");
+ throw new DrillRuntimeException("Failure while checking for impersonation policies.", e);
+ }
+ }
+
+ final String proxyName = session.getCredentials().getUserName();
+ if (!hasImpersonationPrivileges(proxyName, targetName, impersonationPolicies)) {
+ throw UserException.permissionError()
+ .message("Proxy user '%s' is not authorized to impersonate target user '%s'.", proxyName, targetName)
+ .build(logger);
+ }
+
+ // replace session's user credentials
+ final UserCredentials newCredentials = UserCredentials.newBuilder()
+ .setUserName(targetName)
+ .build();
+ session.replaceUserCredentials(this, newCredentials);
+ }
+}
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
index 8ad880a1f..7e90747bb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
@@ -66,6 +66,7 @@ public class UserServer extends BasicServer<RpcType, UserServer.UserClientConnec
final UserWorker worker;
final BufferAllocator alloc;
final UserAuthenticator authenticator;
+ final InboundImpersonationManager impersonationManager;
public UserServer(DrillConfig config, ScanResult classpathScan, BufferAllocator alloc, EventLoopGroup eventLoopGroup,
UserWorker worker, Executor executor) throws DrillbitStartupException {
@@ -80,6 +81,11 @@ public class UserServer extends BasicServer<RpcType, UserServer.UserClientConnec
} else {
authenticator = null;
}
+ if (config.getBoolean(ExecConstants.IMPERSONATION_ENABLED)) {
+ impersonationManager = new InboundImpersonationManager();
+ } else {
+ impersonationManager = null;
+ }
}
@Override
@@ -151,6 +157,10 @@ public class UserServer extends BasicServer<RpcType, UserServer.UserClientConnec
.withUserProperties(inbound.getProperties())
.setSupportComplexTypes(inbound.getSupportComplexTypes())
.build();
+ final String targetName = session.getTargetUserName();
+ if (impersonationManager != null && targetName != null) {
+ impersonationManager.replaceUserOnSession(targetName, session);
+ }
}
public UserSession getSession(){
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserSession.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserSession.java
index e717eaa03..3bf90519a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserSession.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserSession.java
@@ -19,14 +19,16 @@ package org.apache.drill.exec.rpc.user;
import java.util.List;
import java.util.Map;
+import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
+import com.google.common.base.Preconditions;
import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Lists;
import org.apache.calcite.schema.SchemaPlus;
import org.apache.calcite.tools.ValidationException;
-import org.apache.drill.common.exceptions.UserException;
import org.apache.drill.exec.planner.sql.SchemaUtilites;
import org.apache.drill.exec.proto.UserBitShared.UserCredentials;
import org.apache.drill.exec.proto.UserProtos.Property;
@@ -37,14 +39,16 @@ import org.apache.drill.exec.server.options.SessionOptionManager;
import com.google.common.collect.Maps;
public class UserSession {
- static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UserSession.class);
+ private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UserSession.class);
public static final String SCHEMA = "schema";
public static final String USER = "user";
public static final String PASSWORD = "password";
+ public static final String IMPERSONATION_TARGET = "impersonation_target";
+
+ // known property names in lower case
+ private static final Set<String> knownProperties = ImmutableSet.of(SCHEMA, USER, PASSWORD, IMPERSONATION_TARGET);
- private DrillUser user;
- private boolean enableExchanges = true;
private boolean supportComplexTypes = false;
private UserCredentials credentials;
private Map<String, String> properties;
@@ -81,8 +85,13 @@ public class UserSession {
userSession.properties = Maps.newHashMap();
if (properties != null) {
for (int i = 0; i < properties.getPropertiesCount(); i++) {
- Property prop = properties.getProperties(i);
- userSession.properties.put(prop.getKey(), prop.getValue());
+ final Property property = properties.getProperties(i);
+ final String propertyName = property.getKey().toLowerCase();
+ if (knownProperties.contains(propertyName)) {
+ userSession.properties.put(propertyName, property.getValue());
+ } else {
+ logger.warn("Ignoring unknown property: {}", propertyName);
+ }
}
}
return this;
@@ -116,14 +125,28 @@ public class UserSession {
return sessionOptions;
}
- public DrillUser getUser() {
- return user;
- }
-
public UserCredentials getCredentials() {
return credentials;
}
+ /**
+ * Replace current user credentials with the given user's credentials. Meant to be called only by a
+ * {@link InboundImpersonationManager impersonation manager}.
+ *
+ * @param impersonationManager impersonation manager making this call
+ * @param newCredentials user credentials to change to
+ */
+ public void replaceUserCredentials(final InboundImpersonationManager impersonationManager,
+ final UserCredentials newCredentials) {
+ Preconditions.checkNotNull(impersonationManager, "User credentials can only be replaced by an" +
+ " impersonation manager.");
+ credentials = newCredentials;
+ }
+
+ public String getTargetUserName() {
+ return properties.get(IMPERSONATION_TARGET);
+ }
+
public String getDefaultSchemaName() {
return getProp(SCHEMA);
}
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 8b14076bc..1e54e5c02 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
@@ -126,6 +126,7 @@ public class SystemOptionManager extends BaseOptionManager implements AutoClosea
ExecConstants.CTAS_PARTITIONING_HASH_DISTRIBUTE_VALIDATOR,
ExecConstants.ADMIN_USERS_VALIDATOR,
ExecConstants.ADMIN_USER_GROUPS_VALIDATOR,
+ ExecConstants.IMPERSONATION_POLICY_VALIDATOR,
QueryClassLoader.JAVA_COMPILER_VALIDATOR,
QueryClassLoader.JAVA_COMPILER_JANINO_MAXSIZE,
QueryClassLoader.JAVA_COMPILER_DEBUG,
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 38d2797cb..77900433c 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
@@ -130,9 +130,6 @@ public class ImpersonationUtil {
/**
* Create and return proxy user {@link org.apache.hadoop.security.UserGroupInformation} for give user name.
*
- * TODO: we may want to cache the {@link org.apache.hadoop.security.UserGroupInformation} instances as we try to
- * create different instances for the same user which is an unnecessary overhead.
- *
* @param proxyUserName Proxy user name (must be valid)
* @return
*/
@@ -256,4 +253,8 @@ public class ImpersonationUtil {
return false;
}
+
+ // avoid instantiation
+ private ImpersonationUtil() {
+ }
}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/BaseTestImpersonation.java b/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/BaseTestImpersonation.java
index 0d30f3fa8..7fc5cea93 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/BaseTestImpersonation.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/BaseTestImpersonation.java
@@ -114,7 +114,10 @@ public class BaseTestImpersonation extends PlanTestBase {
protected static void startDrillCluster(final boolean isImpersonationEnabled) throws Exception {
final Properties props = cloneDefaultTestConfigProperties();
props.setProperty(ExecConstants.IMPERSONATION_ENABLED, Boolean.toString(isImpersonationEnabled));
+ startDrillCluster(props);
+ }
+ protected static void startDrillCluster(final Properties props) throws Exception {
updateTestCluster(1, DrillConfig.create(props));
}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/TestInboundImpersonation.java b/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/TestInboundImpersonation.java
new file mode 100644
index 000000000..ffda1c04b
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/TestInboundImpersonation.java
@@ -0,0 +1,165 @@
+/**
+ * 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.impersonation;
+
+import com.google.common.collect.Maps;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.dotdrill.DotDrillType;
+import org.apache.drill.exec.proto.UserBitShared;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.user.UserSession;
+import org.apache.drill.exec.rpc.user.security.testing.UserAuthenticatorTestImpl;
+import org.apache.drill.exec.store.dfs.WorkspaceConfig;
+import org.apache.drill.test.UserExceptionMatcher;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.Map;
+import java.util.Properties;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestInboundImpersonation extends BaseTestImpersonation {
+
+ public static final String OWNER = org1Users[0];
+ public static final String OWNER_PASSWORD = "owner";
+
+ public static final String TARGET_NAME = org1Users[1];
+ public static final String TARGET_PASSWORD = "target";
+
+ public static final String DATA_GROUP = org1Groups[0];
+
+ public static final String PROXY_NAME = org1Users[2];
+ public static final String PROXY_PASSWORD = "proxy";
+
+ @BeforeClass
+ public static void setup() throws Exception {
+ startMiniDfsCluster(TestInboundImpersonation.class.getSimpleName());
+ Properties props = cloneDefaultTestConfigProperties();
+ props.setProperty(ExecConstants.IMPERSONATION_ENABLED, Boolean.toString(true));
+ props.setProperty(ExecConstants.USER_AUTHENTICATION_ENABLED, Boolean.toString(true));
+ props.setProperty(ExecConstants.USER_AUTHENTICATOR_IMPL, UserAuthenticatorTestImpl.TYPE);
+
+ startDrillCluster(props);
+ addMiniDfsBasedStorage(createTestWorkspaces());
+ createTestData();
+ }
+
+ private static Map<String, WorkspaceConfig> createTestWorkspaces() throws Exception {
+ Map<String, WorkspaceConfig> workspaces = Maps.newHashMap();
+ createAndAddWorkspace(OWNER, getUserHome(OWNER), (short) 0755, OWNER, DATA_GROUP, workspaces);
+ createAndAddWorkspace(PROXY_NAME, getUserHome(PROXY_NAME), (short) 0755, PROXY_NAME, DATA_GROUP,
+ workspaces);
+ return workspaces;
+ }
+
+ private static void createTestData() throws Exception {
+ // Create table accessible only by OWNER
+ final String tableName = "lineitem";
+ updateClient(OWNER, OWNER_PASSWORD);
+ test("USE " + getWSSchema(OWNER));
+ test(String.format("CREATE TABLE %s as SELECT * FROM cp.`tpch/%s.parquet`;", tableName, tableName));
+
+ // Change the ownership and permissions manually.
+ // Currently there is no option to specify the default permissions and ownership for new tables.
+ final Path tablePath = new Path(getUserHome(OWNER), tableName);
+ fs.setOwner(tablePath, OWNER, DATA_GROUP);
+ fs.setPermission(tablePath, new FsPermission((short) 0700));
+
+ // Create a view on top of lineitem table; allow IMPERSONATION_TARGET to read the view
+ // /user/user0_1 u0_lineitem 750 user0_1:group0_1
+ final String viewName = "u0_lineitem";
+ test(String.format("ALTER SESSION SET `%s`='%o';", ExecConstants.NEW_VIEW_DEFAULT_PERMS_KEY, (short) 0750));
+ test(String.format("CREATE VIEW %s.%s AS SELECT l_orderkey, l_partkey FROM %s.%s;",
+ getWSSchema(OWNER), viewName, getWSSchema(OWNER), "lineitem"));
+ // Verify the view file created has the expected permissions and ownership
+ final Path viewFilePath = new Path(getUserHome(OWNER), viewName + DotDrillType.VIEW.getEnding());
+ final FileStatus status = fs.getFileStatus(viewFilePath);
+ assertEquals(org1Groups[0], status.getGroup());
+ assertEquals(OWNER, status.getOwner());
+ assertEquals((short) 0750, status.getPermission().toShort());
+
+ // Authorize PROXY_NAME to impersonate TARGET_NAME
+ updateClient(UserAuthenticatorTestImpl.PROCESS_USER,
+ UserAuthenticatorTestImpl.PROCESS_USER_PASSWORD);
+ test("ALTER SYSTEM SET `%s`='%s'", ExecConstants.IMPERSONATION_POLICIES_KEY,
+ "[ { proxy_principals : { users: [\"" + PROXY_NAME + "\" ] },"
+ + "target_principals : { users : [\"" + TARGET_NAME + "\"] } } ]");
+ }
+
+ @AfterClass
+ public static void tearDown() throws Exception {
+ updateClient(UserAuthenticatorTestImpl.PROCESS_USER,
+ UserAuthenticatorTestImpl.PROCESS_USER_PASSWORD);
+ test("ALTER SYSTEM RESET `%s`", ExecConstants.IMPERSONATION_POLICIES_KEY);
+ }
+
+ @Test
+ public void selectChainedView() throws Exception {
+ // Connect as PROXY_NAME and query for IMPERSONATION_TARGET
+ // data belongs to OWNER, however a view is shared with IMPERSONATION_TARGET
+ final Properties connectionProps = new Properties();
+ connectionProps.setProperty(UserSession.USER, PROXY_NAME);
+ connectionProps.setProperty(UserSession.PASSWORD, PROXY_PASSWORD);
+ connectionProps.setProperty(UserSession.IMPERSONATION_TARGET, TARGET_NAME);
+ updateClient(connectionProps);
+
+ testBuilder()
+ .sqlQuery("SELECT * FROM %s.u0_lineitem ORDER BY l_orderkey LIMIT 1", getWSSchema(OWNER))
+ .ordered()
+ .baselineColumns("l_orderkey", "l_partkey")
+ .baselineValues(1, 1552)
+ .go();
+ }
+
+ @Test(expected = RpcException.class)
+ // PERMISSION ERROR: Proxy user 'user2_1' is not authorized to impersonate target user 'user0_2'.
+ public void unauthorizedTarget() throws Exception {
+ final String unauthorizedTarget = org2Users[0];
+ final Properties connectionProps = new Properties();
+ connectionProps.setProperty(UserSession.USER, PROXY_NAME);
+ connectionProps.setProperty(UserSession.PASSWORD, PROXY_PASSWORD);
+ connectionProps.setProperty(UserSession.IMPERSONATION_TARGET, unauthorizedTarget);
+ updateClient(connectionProps); // throws up
+ }
+
+ @Test
+ public void invalidPolicy() throws Exception {
+ thrownException.expect(new UserExceptionMatcher(UserBitShared.DrillPBError.ErrorType.VALIDATION,
+ "Invalid impersonation policies."));
+ updateClient(UserAuthenticatorTestImpl.PROCESS_USER,
+ UserAuthenticatorTestImpl.PROCESS_USER_PASSWORD);
+ test("ALTER SYSTEM SET `%s`='%s'", ExecConstants.IMPERSONATION_POLICIES_KEY,
+ "[ invalid json ]");
+ }
+
+ @Test
+ public void invalidProxy() throws Exception {
+ thrownException.expect(new UserExceptionMatcher(UserBitShared.DrillPBError.ErrorType.VALIDATION,
+ "Proxy principals cannot have a wildcard entry."));
+ updateClient(UserAuthenticatorTestImpl.PROCESS_USER,
+ UserAuthenticatorTestImpl.PROCESS_USER_PASSWORD);
+ test("ALTER SYSTEM SET `%s`='%s'", ExecConstants.IMPERSONATION_POLICIES_KEY,
+ "[ { proxy_principals : { users: [\"*\" ] },"
+ + "target_principals : { users : [\"" + TARGET_NAME + "\"] } } ]");
+ }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/TestInboundImpersonationPrivileges.java b/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/TestInboundImpersonationPrivileges.java
new file mode 100644
index 000000000..e5a0148ab
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/impersonation/TestInboundImpersonationPrivileges.java
@@ -0,0 +1,144 @@
+/**
+ * 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.impersonation;
+
+import com.google.common.base.Charsets;
+import com.google.common.io.Files;
+import org.apache.drill.common.util.FileUtils;
+import org.apache.drill.exec.ExecConstants;
+import org.apache.drill.exec.rpc.user.InboundImpersonationManager;
+import org.apache.drill.exec.server.options.OptionValue;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static junit.framework.Assert.assertEquals;
+
+public class TestInboundImpersonationPrivileges extends BaseTestImpersonation {
+ private static final org.slf4j.Logger logger =
+ org.slf4j.LoggerFactory.getLogger(TestInboundImpersonationPrivileges.class);
+
+ // policies on which the tests are based
+ private static final String IMPERSONATION_POLICIES;
+
+ static {
+ try {
+ IMPERSONATION_POLICIES = Files.toString(FileUtils.getResourceAsFile("/inbound_impersonation_policies.json"),
+ Charsets.UTF_8);
+ } catch (final IOException e) {
+ throw new RuntimeException("Cannot load impersonation policies.", e);
+ }
+ }
+
+ private static boolean checkPrivileges(final String proxyName, final String targetName) {
+ ExecConstants.IMPERSONATION_POLICY_VALIDATOR.validate(
+ OptionValue.createString(OptionValue.OptionType.SYSTEM,
+ ExecConstants.IMPERSONATION_POLICIES_KEY,
+ IMPERSONATION_POLICIES));
+ try {
+ return InboundImpersonationManager.hasImpersonationPrivileges(proxyName, targetName, IMPERSONATION_POLICIES);
+ } catch (final Exception e) {
+ logger.error("Failed to check impersonation privileges.", e);
+ return false;
+ }
+ }
+
+ private static void run(final String proxyName, final String targetName, final boolean expected) {
+ assertEquals("proxyName: " + proxyName + " targetName: " + targetName,
+ expected, checkPrivileges(proxyName, targetName));
+ }
+
+ @Test
+ public void allTargetUsers() {
+ for (final String user : org1Users) {
+ run("user0_1", user, true);
+ }
+ for (final String user : org2Users) {
+ run("user0_1", user, true);
+ }
+ }
+
+ @Test
+ public void noTargetUsers() {
+ for (final String user : org1Users) {
+ run("user1_1", user, false);
+ }
+ for (final String user : org2Users) {
+ run("user1_1", user, false);
+ }
+ }
+
+ @Test
+ public void someTargetUsersAndGroups() {
+ run("user2_1", "user3_1", true);
+ run("user2_1", "user3_1", true);
+ run("user2_1", "user1_1", false);
+ run("user2_1", "user4_1", false);
+ for (final String user : org1Users) {
+ if (!user.equals("user3_1") && !user.equals("user2_1")) {
+ run("user2_1", user, false);
+ }
+ }
+ for (final String user : org2Users) {
+ run("user2_1", user, false);
+ }
+ }
+
+ @Test
+ public void someTargetUsers() {
+ run("user4_1", "user1_1", true);
+ run("user4_1", "user3_1", true);
+ for (final String user : org1Users) {
+ if (!user.equals("user1_1") && !user.equals("user3_1")) {
+ run("user4_1", user, false);
+ }
+ }
+ for (final String user : org2Users) {
+ run("user4_1", user, false);
+ }
+ }
+
+ @Test
+ public void oneTargetGroup() {
+ run("user5_1", "user4_2", true);
+ run("user5_1", "user5_2", true);
+ run("user5_1", "user4_1", false);
+ run("user5_1", "user3_2", false);
+ }
+
+ @Test
+ public void twoTargetUsers() {
+ run("user5_2", "user0_2", true);
+ run("user5_2", "user1_2", true);
+ run("user5_2", "user2_2", false);
+ run("user5_2", "user0_1", false);
+ run("user5_2", "user1_1", false);
+ }
+
+ @Test
+ public void twoTargetGroups() {
+ run("user3_2", "user4_2", true);
+ run("user3_2", "user1_2", true);
+ run("user3_2", "user2_2", true);
+ run("user3_2", "user0_2", false);
+ run("user3_2", "user5_2", false);
+ for (final String user : org1Users) {
+ run("user3_2", user, false);
+ }
+ }
+}
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/testing/UserAuthenticatorTestImpl.java b/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/testing/UserAuthenticatorTestImpl.java
index 7cfab5ffd..e65eca51f 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/testing/UserAuthenticatorTestImpl.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/rpc/user/security/testing/UserAuthenticatorTestImpl.java
@@ -18,6 +18,7 @@
package org.apache.drill.exec.rpc.user.security.testing;
import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.exec.impersonation.TestInboundImpersonation;
import org.apache.drill.exec.exception.DrillbitStartupException;
import org.apache.drill.exec.rpc.user.security.UserAuthenticationException;
import org.apache.drill.exec.rpc.user.security.UserAuthenticator;
@@ -27,9 +28,17 @@ import org.apache.hadoop.security.UserGroupInformation;
import java.io.IOException;
-/*
- * Implement {@link org.apache.drill.exec.rpc.user.security.UserAuthenticator} for testing UserAuthenticator and
- * authentication of users from Java client to Drillbit.
+import static org.apache.drill.exec.impersonation.TestInboundImpersonation.PROXY_NAME;
+import static org.apache.drill.exec.impersonation.TestInboundImpersonation.PROXY_PASSWORD;
+import static org.apache.drill.exec.impersonation.TestInboundImpersonation.TARGET_NAME;
+import static org.apache.drill.exec.impersonation.TestInboundImpersonation.TARGET_PASSWORD;
+import static org.apache.drill.exec.impersonation.TestInboundImpersonation.OWNER;
+import static org.apache.drill.exec.impersonation.TestInboundImpersonation.OWNER_PASSWORD;
+
+/**
+ * Implement {@link org.apache.drill.exec.rpc.user.security.UserAuthenticator} for testing:
+ * + UserAuthenticator and authentication of users from Java client to Drillbit.
+ * + {@link TestInboundImpersonation user delegation}.
*/
@UserAuthenticatorTemplate(type = UserAuthenticatorTestImpl.TYPE)
public class UserAuthenticatorTestImpl implements UserAuthenticator {
@@ -65,10 +74,20 @@ public class UserAuthenticatorTestImpl implements UserAuthenticator {
return;
}
- if (!(TEST_USER_1.equals(user) && TEST_USER_1_PASSWORD.equals(password)) &&
+ if (
+ !(PROCESS_USER.equals(user) && PROCESS_USER_PASSWORD.equals(password)) &&
+ /**
+ * Used in {@link org.apache.drill.exec.rpc.user.security.TestCustomUserAuthenticator}
+ */
+ !(TEST_USER_1.equals(user) && TEST_USER_1_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))) {
+ /**
+ * Used in {@link TestInboundImpersonation}
+ */
+ !(OWNER.equals(user) && OWNER_PASSWORD.equals(password)) &&
+ !(TARGET_NAME.equals(user) && TARGET_PASSWORD.equals(password)) &&
+ !(PROXY_NAME.equals(user) && PROXY_PASSWORD.equals(password))) {
throw new UserAuthenticationException();
}
}
diff --git a/exec/java-exec/src/test/resources/inbound_impersonation_policies.json b/exec/java-exec/src/test/resources/inbound_impersonation_policies.json
new file mode 100644
index 000000000..8b5533759
--- /dev/null
+++ b/exec/java-exec/src/test/resources/inbound_impersonation_policies.json
@@ -0,0 +1,14 @@
+[
+ { proxy_principals : { users : [ "user0_1"] },
+ target_principals : { users : ["*"] } },
+ { proxy_principals : { users : [ "user2_1"] },
+ target_principals : { users : ["user3_1"], groups : ["group2_1"] } },
+ { proxy_principals : { users : ["user4_1"] },
+ target_principals : { users : ["user1_1", "user3_1"] } },
+ { proxy_principals : { groups : ["group5_1"] },
+ target_principals : { groups : ["group4_2"] } },
+ { proxy_principals : { groups : ["group5_2"] },
+ target_principals : { users : ["user0_2", "user1_2"] } },
+ { proxy_principals : { users : ["user3_2"] },
+ target_principals : { groups : ["group3_2", "group1_2"] } }
+] \ No newline at end of file