From 2467b2fb81e89c7cfc897e369d6b6beeb5b3b222 Mon Sep 17 00:00:00 2001 From: Hitesh Shah Date: Thu, 7 Aug 2014 11:49:15 -0700 Subject: [PATCH 1/9] TEZ-671. Support View/Modify ACLs for DAGs. --- .../apache/tez/dag/api/TezConfiguration.java | 43 ++ .../org/apache/tez/client/LocalClient.java | 2 +- .../tez/dag/api/client/DAGClientHandler.java | 27 +- ...GClientAMProtocolBlockingPBServerImpl.java | 75 ++++ .../org/apache/tez/dag/app/AppContext.java | 3 + .../org/apache/tez/dag/app/DAGAppMaster.java | 15 + .../java/org/apache/tez/dag/app/dag/DAG.java | 11 +- .../apache/tez/dag/app/dag/impl/DAGImpl.java | 30 +- .../app/security/ACLConfigurationParser.java | 116 ++++++ .../tez/dag/app/security/ACLManager.java | 169 ++++++++ .../apache/tez/dag/app/security/Groups.java | 281 +++++++++++++ .../tez/dag/app/dag/impl/TestDAGImpl.java | 21 +- .../security/TestACLConfigurationParser.java | 67 ++++ .../tez/dag/app/security/TestACLManager.java | 368 ++++++++++++++++++ .../api/impl/TezMergedInputContextImpl.java | 18 + tez-tools/swimlanes/amlogparser.py | 19 + tez-tools/swimlanes/swimlane.py | 19 + 17 files changed, 1245 insertions(+), 39 deletions(-) create mode 100644 tez-dag/src/main/java/org/apache/tez/dag/app/security/ACLConfigurationParser.java create mode 100644 tez-dag/src/main/java/org/apache/tez/dag/app/security/ACLManager.java create mode 100644 tez-dag/src/main/java/org/apache/tez/dag/app/security/Groups.java create mode 100644 tez-dag/src/test/java/org/apache/tez/dag/app/security/TestACLConfigurationParser.java create mode 100644 tez-dag/src/test/java/org/apache/tez/dag/app/security/TestACLManager.java diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java index ce46bf3e62..b832413221 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java @@ -629,4 +629,47 @@ public TezConfiguration(Configuration conf) { */ public static final int TEZ_AM_INLINE_TASK_EXECUTION_MAX_TASKS_DEFAULT = 1; + + // ACLs related configuration + // Format supports a comma-separated list of users and groups with the users and groups separated + // by whitespace. e.g. "user1,user2 group1,group2" + // All users/groups that have access to do operations on the AM also have access to similar + // operations on all DAGs within that AM/session. + // By default, the "owner" i.e. the user who started the session will always have full admin + // access to the AM. Also, the user that submitted the DAG has full admin access to all operations + // on that DAG. + // + // If no value is specified or an invalid configuration is specified, + // only the user who submitted the AM and/or DAG can do the appropriate operations. + // For example, "user1,user2 group1, group2" is an invalid configuration value as splitting by + // whitespace produces 3 lists instead of 2. + + // If the value specified is "*", all users are allowed to do the operation. + + /** + * Configuration to enable/disable ACL checks. + */ + public static final String TEZ_AM_ACLS_ENABLED = TEZ_AM_PREFIX + "acls.enabled"; + public static final boolean TEZ_AM_ACLS_ENABLED_DEFAULT = true; + + /** + * AM view ACLs. This allows the specified users/groups to view the status of the AM and all DAGs + * that run within this AM. + */ + public static final String TEZ_AM_VIEW_ACLS = TEZ_AM_PREFIX + "view-acls"; + /** + * AM modify ACLs. This allows the specified users/groups to run modify operations on the AM + * such as submitting DAGs, pre-warming the session, killing DAGs or shutting down the session. + */ + public static final String TEZ_AM_MODIFY_ACLS = TEZ_AM_PREFIX + "modify-acls"; + /** + * DAG view ACLs. This allows the specified users/groups to view the status of the given DAG. + */ + public static final String TEZ_DAG_VIEW_ACLS = TEZ_AM_PREFIX + "dag.view-acls"; + /** + * DAG modify ACLs. This allows the specified users/groups to run modify operations on the DAG + * such as killing the DAG. + */ + public static final String TEZ_DAG_MODIFY_ACLS = TEZ_AM_PREFIX + "dag.modify-acls"; + } diff --git a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java index 1a86a2b597..84bcb311ef 100644 --- a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java +++ b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java @@ -7,7 +7,7 @@ * "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 + * 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, diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGClientHandler.java b/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGClientHandler.java index 6dca990fd9..80a5b6eb29 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGClientHandler.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGClientHandler.java @@ -26,15 +26,13 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.yarn.api.records.LocalResource; -import org.apache.hadoop.yarn.event.Dispatcher; import org.apache.tez.client.PreWarmContext; import org.apache.tez.client.TezAppMasterStatus; import org.apache.tez.dag.api.TezException; import org.apache.tez.dag.api.records.DAGProtos.DAGPlan; import org.apache.tez.dag.app.DAGAppMaster; import org.apache.tez.dag.app.dag.DAG; -import org.apache.tez.dag.app.dag.event.DAGEvent; -import org.apache.tez.dag.app.dag.event.DAGEventType; +import org.apache.tez.dag.app.security.ACLManager; import org.apache.tez.dag.records.TezDAGID; public class DAGClientHandler { @@ -141,4 +139,27 @@ public synchronized void preWarmContainers(PreWarmContext preWarmContext) dagAppMaster.startPreWarmContainers(preWarmContext); } + public ACLManager getACLManager() { + return dagAppMaster.getACLManager(); + } + + public ACLManager getACLManager(String dagIdStr) throws TezException { + TezDAGID dagId = TezDAGID.fromString(dagIdStr); + if (dagId == null) { + throw new TezException("Bad dagId: " + dagIdStr); + } + DAG currentDAG = getCurrentDAG(); + if (currentDAG == null) { + throw new TezException("No running dag at present"); + } + if (!currentDAG.getID().toString().equals(dagId.toString())) { + LOG.warn("Current DAGID : " + + (currentDAG.getID() == null ? "NULL" : currentDAG.getID()) + + ", Looking for string (not found): " + dagIdStr + ", dagIdObj: " + + dagId); + throw new TezException("Unknown dagId: " + dagIdStr); + } + return currentDAG.getACLManager(); + } + } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java index d47bff0fb0..4e39fd3e33 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java @@ -18,9 +18,12 @@ package org.apache.tez.dag.api.client.rpc; +import java.io.IOException; +import java.security.AccessControlException; import java.util.List; import java.util.Map; +import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.yarn.api.records.LocalResource; import org.apache.tez.client.TezAppMasterStatus; import org.apache.tez.dag.api.DagTypeConverters; @@ -62,6 +65,15 @@ public DAGClientAMProtocolBlockingPBServerImpl(DAGClientHandler real) { @Override public GetAllDAGsResponseProto getAllDAGs(RpcController controller, GetAllDAGsRequestProto request) throws ServiceException { + String user; + try { + user = UserGroupInformation.getCurrentUser().getShortUserName(); + } catch (IOException e) { + throw wrapException(e); + } + if (!real.getACLManager().checkAMViewAccess(user)) { + throw new AccessControlException("User " + user + " cannot perform AM view operation"); + } try{ List dagIds = real.getAllDAGs(); return GetAllDAGsResponseProto.newBuilder().addAllDagId(dagIds).build(); @@ -73,8 +85,17 @@ public GetAllDAGsResponseProto getAllDAGs(RpcController controller, @Override public GetDAGStatusResponseProto getDAGStatus(RpcController controller, GetDAGStatusRequestProto request) throws ServiceException { + String user; + try { + user = UserGroupInformation.getCurrentUser().getShortUserName(); + } catch (IOException e) { + throw wrapException(e); + } try { String dagId = request.getDagId(); + if (!real.getACLManager(dagId).checkDAGViewAccess(user)) { + throw new AccessControlException("User " + user + " cannot perform DAG view operation"); + } DAGStatus status; status = real.getDAGStatus(dagId, DagTypeConverters.convertStatusGetOptsFromProto( @@ -91,8 +112,17 @@ public GetDAGStatusResponseProto getDAGStatus(RpcController controller, @Override public GetVertexStatusResponseProto getVertexStatus(RpcController controller, GetVertexStatusRequestProto request) throws ServiceException { + String user; + try { + user = UserGroupInformation.getCurrentUser().getShortUserName(); + } catch (IOException e) { + throw wrapException(e); + } try { String dagId = request.getDagId(); + if (!real.getACLManager(dagId).checkDAGViewAccess(user)) { + throw new AccessControlException("User " + user + " cannot perform DAG view operation"); + } String vertexName = request.getVertexName(); VertexStatus status = real.getVertexStatus(dagId, vertexName, DagTypeConverters.convertStatusGetOptsFromProto( @@ -109,8 +139,17 @@ public GetVertexStatusResponseProto getVertexStatus(RpcController controller, @Override public TryKillDAGResponseProto tryKillDAG(RpcController controller, TryKillDAGRequestProto request) throws ServiceException { + String user; + try { + user = UserGroupInformation.getCurrentUser().getShortUserName(); + } catch (IOException e) { + throw wrapException(e); + } try { String dagId = request.getDagId(); + if (!real.getACLManager(dagId).checkDAGModifyAccess(user)) { + throw new AccessControlException("User " + user + " cannot perform DAG modify operation"); + } real.tryKillDAG(dagId); return TryKillDAGResponseProto.newBuilder().build(); } catch (TezException e) { @@ -121,6 +160,15 @@ public TryKillDAGResponseProto tryKillDAG(RpcController controller, @Override public SubmitDAGResponseProto submitDAG(RpcController controller, SubmitDAGRequestProto request) throws ServiceException { + String user; + try { + user = UserGroupInformation.getCurrentUser().getShortUserName(); + } catch (IOException e) { + throw wrapException(e); + } + if (!real.getACLManager().checkAMModifyAccess(user)) { + throw new AccessControlException("User " + user + " cannot perform AM modify operation"); + } try{ DAGPlan dagPlan = request.getDAGPlan(); Map additionalResources = null; @@ -142,6 +190,15 @@ ServiceException wrapException(Exception e){ @Override public ShutdownSessionResponseProto shutdownSession(RpcController arg0, ShutdownSessionRequestProto arg1) throws ServiceException { + String user; + try { + user = UserGroupInformation.getCurrentUser().getShortUserName(); + } catch (IOException e) { + throw wrapException(e); + } + if (!real.getACLManager().checkAMModifyAccess(user)) { + throw new AccessControlException("User " + user + " cannot perform AM modify operation"); + } real.shutdownAM(); return ShutdownSessionResponseProto.newBuilder().build(); } @@ -149,6 +206,15 @@ public ShutdownSessionResponseProto shutdownSession(RpcController arg0, @Override public GetAMStatusResponseProto getAMStatus(RpcController controller, GetAMStatusRequestProto request) throws ServiceException { + String user; + try { + user = UserGroupInformation.getCurrentUser().getShortUserName(); + } catch (IOException e) { + throw wrapException(e); + } + if (!real.getACLManager().checkAMViewAccess(user)) { + throw new AccessControlException("User " + user + " cannot perform AM view operation"); + } try { TezAppMasterStatus sessionStatus = real.getSessionStatus(); return GetAMStatusResponseProto.newBuilder().setStatus( @@ -163,6 +229,15 @@ public GetAMStatusResponseProto getAMStatus(RpcController controller, public DAGClientAMProtocolRPC.PreWarmResponseProto preWarm( RpcController controller, PreWarmRequestProto request) throws ServiceException { + String user; + try { + user = UserGroupInformation.getCurrentUser().getShortUserName(); + } catch (IOException e) { + throw wrapException(e); + } + if (!real.getACLManager().checkAMModifyAccess(user)) { + throw new AccessControlException("User " + user + " cannot perform AM modify operation"); + } try { real.preWarmContainers( DagTypeConverters.convertPreWarmContextFromProto( diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java b/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java index 89fa1a6a66..fc76cdc947 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java @@ -33,6 +33,7 @@ import org.apache.tez.dag.app.rm.TaskSchedulerEventHandler; import org.apache.tez.dag.app.rm.container.AMContainerMap; import org.apache.tez.dag.app.rm.node.AMNodeMap; +import org.apache.tez.dag.app.security.ACLManager; import org.apache.tez.dag.history.HistoryEventHandler; import org.apache.tez.dag.records.TezDAGID; @@ -90,4 +91,6 @@ public interface AppContext { boolean isRecoveryEnabled(); + ACLManager getAMACLManager(); + } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java index 0626fc841e..e13a10c23f 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java @@ -141,6 +141,8 @@ import org.apache.tez.dag.app.rm.container.ContainerSignatureMatcher; import org.apache.tez.dag.app.rm.node.AMNodeEventType; import org.apache.tez.dag.app.rm.node.AMNodeMap; +import org.apache.tez.dag.app.security.ACLManager; +import org.apache.tez.dag.app.security.Groups; import org.apache.tez.dag.history.DAGHistoryEvent; import org.apache.tez.dag.history.HistoryEventHandler; import org.apache.tez.dag.history.events.AMLaunchedEvent; @@ -258,6 +260,7 @@ public class DAGAppMaster extends AbstractService { private final AtomicInteger successfulDAGs = new AtomicInteger(); private final AtomicInteger failedDAGs = new AtomicInteger(); private final AtomicInteger killedDAGs = new AtomicInteger(); + private ACLManager aclManager; // must be LinkedHashMap to preserve order of service addition Map services = @@ -311,6 +314,9 @@ public synchronized void serviceInit(final Configuration conf) throws Exception dispatcher = createDispatcher(); context = new RunningAppContext(conf); + Groups userGroupMapping = new Groups(this.amConf); + this.aclManager = new ACLManager(userGroupMapping, appMasterUgi.getShortUserName(), + this.amConf); clientHandler = new DAGClientHandler(this); @@ -562,6 +568,10 @@ public void setCurrentDAG(DAG currentDAG) { context.setDAG(currentDAG); } + public ACLManager getACLManager() { + return this.aclManager; + } + private class DAGAppMasterEventHandler implements EventHandler { @Override @@ -1253,6 +1263,11 @@ public boolean isRecoveryEnabled() { return recoveryEnabled; } + @Override + public ACLManager getAMACLManager() { + return aclManager; + } + @Override public Map getApplicationACLs() { if (getServiceState() != STATE.STARTED) { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java index 45fb50a629..4ef19966e1 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java @@ -31,6 +31,7 @@ import org.apache.tez.dag.api.client.StatusGetOpts; import org.apache.tez.dag.api.client.VertexStatusBuilder; import org.apache.tez.dag.api.records.DAGProtos.DAGPlan; +import org.apache.tez.dag.app.security.ACLManager; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.records.TezDAGID; import org.apache.tez.dag.records.TezVertexID; @@ -75,16 +76,12 @@ VertexStatusBuilder getVertexStatus(String vertexName, boolean isComplete(); - /** - * @return the ACLs for this job for each type of JobACL given. - */ - Map getJobACLs(); - - boolean checkAccess(UserGroupInformation callerUGI, ApplicationAccessType jobOperation); - Credentials getCredentials(); UserGroupInformation getDagUGI(); DAGState restoreFromEvent(HistoryEvent historyEvent); + + ACLManager getACLManager(); + } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java index 614bf98798..4103e2c3a8 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java @@ -43,9 +43,7 @@ import org.apache.hadoop.security.Credentials; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.StringUtils; -import org.apache.hadoop.yarn.api.records.ApplicationAccessType; import org.apache.hadoop.yarn.event.EventHandler; -import org.apache.hadoop.yarn.server.security.ApplicationACLsManager; import org.apache.hadoop.yarn.state.InvalidStateTransitonException; import org.apache.hadoop.yarn.state.MultipleArcTransition; import org.apache.hadoop.yarn.state.SingleArcTransition; @@ -95,6 +93,7 @@ import org.apache.tez.dag.app.dag.event.VertexEventRecoverVertex; import org.apache.tez.dag.app.dag.event.VertexEventTermination; import org.apache.tez.dag.app.dag.event.VertexEventType; +import org.apache.tez.dag.app.security.ACLManager; import org.apache.tez.dag.history.DAGHistoryEvent; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.events.DAGCommitStartedEvent; @@ -129,7 +128,6 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG, //final fields private final TezDAGID dagId; private final Clock clock; - private final ApplicationACLsManager aclsManager; // TODO Recovery //private final List amInfos; @@ -153,6 +151,7 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG, private final String userName; private final AppContext appContext; private final UserGroupInformation dagUGI; + private final ACLManager aclManager; volatile Map vertices = new HashMap(); private Map edges = new HashMap(); @@ -427,7 +426,8 @@ public DAGImpl(TezDAGID dagId, dagUGI.addCredentials(this.credentials); } - this.aclsManager = new ApplicationACLsManager(conf); + this.aclManager = new ACLManager(appContext.getAMACLManager(), dagUGI.getShortUserName(), + this.conf); this.profilerOptions = new JavaProfilerOptions(conf); // This "this leak" is okay because the retained pointer is in an @@ -459,13 +459,6 @@ EventHandler getEventHandler() { return this.eventHandler; } - @Override - public boolean checkAccess(UserGroupInformation callerUGI, - ApplicationAccessType jobOperation) { - return aclsManager.checkAccess(callerUGI, jobOperation, userName, - this.dagId.getApplicationId()); - } - @Override public Vertex getVertex(TezVertexID vertexID) { readLock.lock(); @@ -534,6 +527,11 @@ public DAGState restoreFromEvent(HistoryEvent historyEvent) { } } + @Override + public ACLManager getACLManager() { + return this.aclManager; + } + @Override public TezCounters getAllCounters() { @@ -1134,16 +1132,6 @@ DAGTerminationCause getTerminationCause() { } } - /* - * (non-Javadoc) - * @see org.apache.hadoop.mapreduce.v2.app2.job.Job#getJobACLs() - */ - @Override - public Map getJobACLs() { - // TODO ApplicationACLs - return null; - } - public DAGState initializeDAG() { return initializeDAG(null); } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/security/ACLConfigurationParser.java b/tez-dag/src/main/java/org/apache/tez/dag/app/security/ACLConfigurationParser.java new file mode 100644 index 0000000000..c72561e9ac --- /dev/null +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/security/ACLConfigurationParser.java @@ -0,0 +1,116 @@ +/** + * 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.tez.dag.app.security; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.regex.Pattern; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.util.StringUtils; +import org.apache.tez.dag.api.TezConfiguration; +import org.apache.tez.dag.app.security.ACLManager.ACLType; + +import com.google.common.collect.Sets; + +public class ACLConfigurationParser { + + private static final Log LOG = LogFactory.getLog(ACLConfigurationParser.class); + + private final Configuration conf; + private final Map> allowedUsers; + private final Map> allowedGroups; + private static final Pattern splitPattern = Pattern.compile("\\s+"); + + public ACLConfigurationParser(Configuration conf) { + this(conf, false); + } + + public ACLConfigurationParser(Configuration conf, boolean dagACLs) { + this.conf = conf; + allowedUsers = new HashMap>(2); + allowedGroups = new HashMap>(2); + parse(dagACLs); + } + + + private void parse(boolean dagACLs) { + if (!dagACLs) { + parseACLType(TezConfiguration.TEZ_AM_VIEW_ACLS, ACLType.AM_VIEW_ACL); + parseACLType(TezConfiguration.TEZ_AM_MODIFY_ACLS, ACLType.AM_MODIFY_ACL); + } else { + parseACLType(TezConfiguration.TEZ_DAG_VIEW_ACLS, ACLType.DAG_VIEW_ACL); + parseACLType(TezConfiguration.TEZ_DAG_MODIFY_ACLS, ACLType.DAG_MODIFY_ACL); + } + } + + private void parseACLType(String configProperty, ACLType aclType) { + String aclsStr = conf.get(configProperty); + if (aclsStr == null || aclsStr.isEmpty()) { + return; + } + final String[] splits = splitPattern.split(aclsStr); + int counter = -1; + String userListStr = null; + String groupListStr = null; + for (String s : splits) { + if (s.isEmpty()) { + continue; + } + ++counter; + if (counter == 0) { + userListStr = s; + } else if (counter == 1) { + groupListStr = s; + } else { + LOG.warn("Invalid configuration specified for " + configProperty + + ", ignoring configured ACLs, value=" + aclsStr); + return; + } + } + if (userListStr == null) { + return; + } + if (userListStr.equals(ACLManager.WILDCARD_ACL_VALUE)) { + allowedUsers.put(aclType, Sets.newHashSet(ACLManager.WILDCARD_ACL_VALUE)); + } else { + if (userListStr.length() >= 1) { + allowedUsers.put(aclType, + Sets.newHashSet(StringUtils.getTrimmedStringCollection(userListStr))); + } + if (groupListStr != null && groupListStr.length() >= 1) { + allowedGroups.put(aclType, + Sets.newHashSet(StringUtils.getTrimmedStringCollection(groupListStr))); + } + } + } + + public Map> getAllowedUsers() { + return Collections.unmodifiableMap(allowedUsers); + } + + public Map> getAllowedGroups() { + return Collections.unmodifiableMap(allowedGroups); + } + +} diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/security/ACLManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/security/ACLManager.java new file mode 100644 index 0000000000..ec3b51cd4f --- /dev/null +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/security/ACLManager.java @@ -0,0 +1,169 @@ +/** + * 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.tez.dag.app.security; + +import java.io.IOException; +import java.util.EnumSet; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.tez.dag.api.TezConfiguration; + +import com.google.common.annotations.VisibleForTesting; + +/** + * Class to manage ACLs for the Tez AM and DAGs and provides functionality to check whether + * a user is authorized to take certain actions. + */ +public class ACLManager { + + private static final Log LOG = LogFactory.getLog(ACLManager.class); + static final String WILDCARD_ACL_VALUE = "*"; + + static enum ACLType { + AM_VIEW_ACL, + AM_MODIFY_ACL, + DAG_VIEW_ACL, + DAG_MODIFY_ACL + } + + private final String dagUser; + private final String amUser; + private final Map> users; + private final Map> groups; + private final Groups userGroupMapping; + private final boolean aclsEnabled; + + public ACLManager(Groups userGroupMapping, String amUser) { + this(userGroupMapping, amUser, null); + } + + public ACLManager(Groups userGroupMapping, String amUser, Configuration conf) { + this.userGroupMapping = userGroupMapping; + this.amUser = amUser; + this.dagUser = null; + this.users = new HashMap>(); + this.groups = new HashMap>(); + if (conf != null) { + aclsEnabled = conf.getBoolean(TezConfiguration.TEZ_AM_ACLS_ENABLED, + TezConfiguration.TEZ_AM_ACLS_ENABLED_DEFAULT); + if (!aclsEnabled) { + return; + } + ACLConfigurationParser parser = new ACLConfigurationParser(conf); + if (parser.getAllowedUsers() != null) { + this.users.putAll(parser.getAllowedUsers()); + } + if (parser.getAllowedGroups() != null) { + this.groups.putAll(parser.getAllowedGroups()); + } + } else { + aclsEnabled = true; + } + } + + public ACLManager(ACLManager amACLManager, String dagUser, Configuration dagConf) { + this.userGroupMapping = amACLManager.userGroupMapping; + this.amUser = amACLManager.amUser; + this.dagUser = dagUser; + this.users = amACLManager.users; + this.groups = amACLManager.groups; + this.aclsEnabled = amACLManager.aclsEnabled; + if (!aclsEnabled) { + return; + } + if (dagConf != null) { + ACLConfigurationParser parser = new ACLConfigurationParser(dagConf, true); + if (parser.getAllowedUsers() != null) { + this.users.putAll(parser.getAllowedUsers()); + } + if (parser.getAllowedGroups() != null) { + this.groups.putAll(parser.getAllowedGroups()); + } + } + } + + @VisibleForTesting + boolean checkAccess(String user, ACLType aclType) { + if (!aclsEnabled) { + return true; + } + if (amUser.equals(user)) { + return true; + } + if (EnumSet.of(ACLType.DAG_MODIFY_ACL, ACLType.DAG_VIEW_ACL).contains(aclType)) { + if (dagUser != null && dagUser.equals(user)) { + return true; + } + } + if (users != null && !users.isEmpty()) { + Set set = users.get(aclType); + if (set != null) { + if (set.contains(WILDCARD_ACL_VALUE)) { + return true; + } + if (set.contains(user)) { + return true; + } + } + } + if (groups != null && !groups.isEmpty()) { + try { + Set set = groups.get(aclType); + if (set != null) { + Set userGrps = userGroupMapping.getGroups(user); + for (String userGrp : userGrps) { + if (set.contains(userGrp)) { + return true; + } + } + } + } catch (IOException e) { + LOG.warn("Failed to retrieve groups for user" + + ", user=" + user, e); + } + } + return false; + } + + public boolean checkAMViewAccess(String user) { + return checkAccess(user, ACLType.AM_VIEW_ACL); + } + + public boolean checkAMModifyAccess(String user) { + return checkAccess(user, ACLType.AM_MODIFY_ACL); + } + + public boolean checkDAGViewAccess(String user) { + return checkAccess(user, ACLType.AM_VIEW_ACL) + || checkAccess(user, ACLType.DAG_VIEW_ACL); + } + + public boolean checkDAGModifyAccess(String user) { + return checkAccess(user, ACLType.AM_MODIFY_ACL) + || checkAccess(user, ACLType.DAG_MODIFY_ACL); + } + +} + + diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/security/Groups.java b/tez-dag/src/main/java/org/apache/tez/dag/app/security/Groups.java new file mode 100644 index 0000000000..624156c0d6 --- /dev/null +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/security/Groups.java @@ -0,0 +1,281 @@ +/** + * 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.tez.dag.app.security; + +import java.io.IOException; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; + +import org.apache.hadoop.HadoopIllegalArgumentException; +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeys; +import org.apache.hadoop.security.GroupMappingServiceProvider; +import org.apache.hadoop.security.ShellBasedUnixGroupsMapping; +import org.apache.hadoop.util.ReflectionUtils; +import org.apache.hadoop.util.StringUtils; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.tez.dag.api.TezUncheckedException; + +import com.google.common.collect.Sets; + +/** + * A user-to-groups mapping service. + * + * {@link Groups} allows for server to get the various group memberships + * of a given user via the {@link #getGroups(String)} call, thus ensuring + * a consistent user-to-groups mapping and protects against vagaries of + * different mappings on servers and clients in a Hadoop cluster. + */ +@Private +public class Groups { + private static final Log LOG = LogFactory.getLog(Groups.class); + + private final GroupMappingServiceProvider impl; + + private final Map userToGroupsMap = + new ConcurrentHashMap(); + private final Map> staticUserToGroupsMap = + new HashMap>(); + private final long cacheTimeout; + private final long warningDeltaMs; + + public Groups(Configuration conf) { + impl = + ReflectionUtils.newInstance( + conf.getClass(CommonConfigurationKeys.HADOOP_SECURITY_GROUP_MAPPING, + ShellBasedUnixGroupsMapping.class, + GroupMappingServiceProvider.class), + conf); + + cacheTimeout = + conf.getLong(CommonConfigurationKeys.HADOOP_SECURITY_GROUPS_CACHE_SECS, + CommonConfigurationKeys.HADOOP_SECURITY_GROUPS_CACHE_SECS_DEFAULT) * 1000; + warningDeltaMs = + conf.getLong(CommonConfigurationKeys.HADOOP_SECURITY_GROUPS_CACHE_WARN_AFTER_MS, + CommonConfigurationKeys.HADOOP_SECURITY_GROUPS_CACHE_WARN_AFTER_MS_DEFAULT); + parseStaticMapping(conf); + + if (cacheTimeout < 0 || warningDeltaMs <= 0) { + String message = "Invalid values for configuring Groups cache" + + ", cacheTimeout=" + cacheTimeout + + ", warningDeltaTimeMs=" + warningDeltaMs; + LOG.warn(message); + throw new TezUncheckedException(message); + } + + if(LOG.isDebugEnabled()) + LOG.debug("Group mapping impl=" + impl.getClass().getName() + + "; cacheTimeout=" + cacheTimeout + "; warningDeltaMs=" + + warningDeltaMs); + } + + /* + * Parse the hadoop.user.group.static.mapping.overrides configuration to + * staticUserToGroupsMap + */ + private void parseStaticMapping(Configuration conf) { + String staticMapping = conf.get( + CommonConfigurationKeys.HADOOP_USER_GROUP_STATIC_OVERRIDES, + CommonConfigurationKeys.HADOOP_USER_GROUP_STATIC_OVERRIDES_DEFAULT); + Collection mappings = StringUtils.getStringCollection( + staticMapping, ";"); + for (String users : mappings) { + Collection userToGroups = StringUtils.getStringCollection(users, + "="); + if (userToGroups.size() < 1 || userToGroups.size() > 2) { + throw new HadoopIllegalArgumentException("Configuration " + + CommonConfigurationKeys.HADOOP_USER_GROUP_STATIC_OVERRIDES + + " is invalid"); + } + String[] userToGroupsArray = userToGroups.toArray(new String[userToGroups + .size()]); + String user = userToGroupsArray[0]; + Set groups = Sets.newHashSet(); + if (userToGroupsArray.length == 2) { + groups.addAll(StringUtils.getStringCollection(userToGroupsArray[1])); + } + staticUserToGroupsMap.put(user, groups); + } + } + + /** + * Determine whether the CachedGroups is expired. + * @param groups cached groups for one user. + * @return true if groups is expired from useToGroupsMap. + */ + private boolean hasExpired(CachedGroups groups, long startMs) { + if (groups == null) { + return true; + } + long timeout = cacheTimeout; + return groups.getTimestamp() + timeout <= startMs; + } + + /** + * Get the group memberships of a given user. + * @param user User's name + * @return the group memberships of the user + * @throws IOException + */ + public Set getGroups(String user) throws IOException { + // No need to lookup for groups of static users + Set staticMapping = staticUserToGroupsMap.get(user); + if (staticMapping != null) { + return staticMapping; + } + // Return cached value if available + CachedGroups groups = userToGroupsMap.get(user); + long startMs = System.currentTimeMillis(); + if (!hasExpired(groups, startMs)) { + if(LOG.isDebugEnabled()) { + LOG.debug("Returning cached groups for '" + user + "'"); + } + if (groups.getGroups().isEmpty()) { + // Even with enabling negative cache, getGroups() has the same behavior + // that throws IOException if the groups for the user is empty. + throw new IOException("No groups found for user " + user); + } + return groups.getGroups(); + } + + // Create and cache user's groups + Set groupList = Sets.newHashSet(); + groupList.addAll(impl.getGroups(user)); + long endMs = System.currentTimeMillis(); + long deltaMs = endMs - startMs; + if (deltaMs > warningDeltaMs) { + LOG.warn("Potential performance problem: getGroups(user=" + user +") " + + "took " + deltaMs + " milliseconds."); + } + groups = new CachedGroups(groupList, endMs); + if (groups.getGroups().isEmpty()) { + throw new IOException("No groups found for user " + user); + } + userToGroupsMap.put(user, groups); + if(LOG.isDebugEnabled()) { + LOG.debug("Returning fetched groups for '" + user + "'"); + } + return groups.getGroups(); + } + + /** + * Refresh all user-to-groups mappings. + */ + public void refresh() { + LOG.info("clearing userToGroupsMap cache"); + try { + impl.cacheGroupsRefresh(); + } catch (IOException e) { + LOG.warn("Error refreshing groups cache", e); + } + userToGroupsMap.clear(); + } + + /** + * Add groups to cache + * + * @param groups list of groups to add to cache + */ + public void cacheGroupsAdd(List groups) { + try { + impl.cacheGroupsAdd(groups); + } catch (IOException e) { + LOG.warn("Error caching groups", e); + } + } + + /** + * Class to hold the cached groups + */ + private static class CachedGroups { + final long timestamp; + final Set groups; + + /** + * Create and initialize group cache + */ + CachedGroups(Set groups, long timestamp) { + this.groups = groups; + this.timestamp = timestamp; + } + + /** + * Returns time of last cache update + * + * @return time of last cache update + */ + public long getTimestamp() { + return timestamp; + } + + /** + * Get set of cached groups + * + * @return cached groups + */ + public Set getGroups() { + return groups; + } + } + + private static Groups GROUPS = null; + + /** + * Get the groups being used to map user-to-groups. + * @return the groups being used to map user-to-groups. + */ + public static Groups getUserToGroupsMappingService() { + return getUserToGroupsMappingService(new Configuration()); + } + + /** + * Get the groups being used to map user-to-groups. + * @param conf + * @return the groups being used to map user-to-groups. + */ + public static synchronized Groups getUserToGroupsMappingService( + Configuration conf) { + + if(GROUPS == null) { + if(LOG.isDebugEnabled()) { + LOG.debug(" Creating new Groups object"); + } + GROUPS = new Groups(conf); + } + return GROUPS; + } + + /** + * Create new groups used to map user-to-groups with loaded configuration. + * @param conf + * @return the groups being used to map user-to-groups. + */ + public static synchronized Groups getUserToGroupsMappingServiceWithLoadedConfiguration( + Configuration conf) { + + GROUPS = new Groups(conf); + return GROUPS; + } +} diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java index 6cb5d566f2..f8454624f1 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java @@ -91,6 +91,8 @@ import org.apache.tez.dag.app.dag.event.VertexEventTaskReschedule; import org.apache.tez.dag.app.dag.event.VertexEventType; import org.apache.tez.dag.app.dag.impl.TestVertexImpl.CountingOutputCommitter; +import org.apache.tez.dag.app.security.ACLManager; +import org.apache.tez.dag.app.security.Groups; import org.apache.tez.dag.history.HistoryEventHandler; import org.apache.tez.dag.records.TezDAGID; import org.apache.tez.dag.records.TezTaskID; @@ -114,6 +116,7 @@ public class TestDAGImpl { private DrainDispatcher dispatcher; private Credentials fsTokens; private AppContext appContext; + private ACLManager aclManager; private ApplicationAttemptId appAttemptId; private DAGImpl dag; private TaskEventDispatcher taskEventDispatcher; @@ -328,7 +331,7 @@ private DAGPlan createTestMRRDAGPlan() { return dag; } - + public static class TotalCountingOutputCommitter extends CountingOutputCommitter { static int totalCommitCounter = 0; public TotalCountingOutputCommitter(OutputCommitterContext context) { @@ -340,7 +343,7 @@ public void commitOutput() throws IOException { super.commitOutput(); } } - + // Create a plan with 3 vertices: A, B, C. Group(A,B)->C private DAGPlan createGroupDAGPlan() { LOG.info("Setting up group dag plan"); @@ -355,7 +358,7 @@ private DAGPlan createGroupDAGPlan() { org.apache.tez.dag.api.Vertex v3 = new org.apache.tez.dag.api.Vertex("vertex3", new ProcessorDescriptor("Processor"), dummyTaskCount, dummyTaskResource); - + DAG dag = new DAG("testDag"); String groupName1 = "uv12"; OutputCommitterDescriptor ocd = new OutputCommitterDescriptor( @@ -364,14 +367,14 @@ private DAGPlan createGroupDAGPlan() { OutputDescriptor outDesc = new OutputDescriptor("output.class"); uv12.addDataSink("uvOut", new DataSinkDescriptor(outDesc, ocd, null)); v3.addDataSink("uvOut", new DataSinkDescriptor(outDesc, ocd, null)); - + GroupInputEdge e1 = new GroupInputEdge(uv12, v3, - new EdgeProperty(DataMovementType.SCATTER_GATHER, + new EdgeProperty(DataMovementType.SCATTER_GATHER, DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, new OutputDescriptor("dummy output class"), - new InputDescriptor("dummy input class")), + new InputDescriptor("dummy input class")), new InputDescriptor("merge.class")); - + dag.addVertex(v1); dag.addVertex(v2); dag.addVertex(v3); @@ -612,16 +615,19 @@ public void setup() { fsTokens = new Credentials(); appContext = mock(AppContext.class); historyEventHandler = mock(HistoryEventHandler.class); + aclManager = new ACLManager(mock(Groups.class), "amUser"); doReturn(conf).when(appContext).getAMConf(); doReturn(appAttemptId).when(appContext).getApplicationAttemptId(); doReturn(appAttemptId.getApplicationId()).when(appContext).getApplicationID(); doReturn(dagId).when(appContext).getCurrentDAGID(); doReturn(historyEventHandler).when(appContext).getHistoryHandler(); + doReturn(aclManager).when(appContext).getAMACLManager(); dag = new DAGImpl(dagId, conf, dagPlan, dispatcher.getEventHandler(), taskAttemptListener, fsTokens, clock, "user", thh, appContext); doReturn(dag).when(appContext).getCurrentDAG(); mrrAppContext = mock(AppContext.class); + doReturn(aclManager).when(mrrAppContext).getAMACLManager(); mrrDagId = TezDAGID.getInstance(appAttemptId.getApplicationId(), 2); mrrDagPlan = createTestMRRDAGPlan(); mrrDag = new DAGImpl(mrrDagId, conf, mrrDagPlan, @@ -634,6 +640,7 @@ public void setup() { doReturn(appAttemptId.getApplicationId()).when(mrrAppContext).getApplicationID(); doReturn(historyEventHandler).when(mrrAppContext).getHistoryHandler(); groupAppContext = mock(AppContext.class); + doReturn(aclManager).when(groupAppContext).getAMACLManager(); groupDagId = TezDAGID.getInstance(appAttemptId.getApplicationId(), 3); groupDagPlan = createGroupDAGPlan(); groupDag = new DAGImpl(groupDagId, conf, groupDagPlan, diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/security/TestACLConfigurationParser.java b/tez-dag/src/test/java/org/apache/tez/dag/app/security/TestACLConfigurationParser.java new file mode 100644 index 0000000000..1070d606ce --- /dev/null +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/security/TestACLConfigurationParser.java @@ -0,0 +1,67 @@ +/** + * 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.tez.dag.app.security; + +import org.apache.hadoop.conf.Configuration; +import org.apache.tez.dag.api.TezConfiguration; +import org.apache.tez.dag.app.security.ACLManager.ACLType; +import org.junit.Assert; +import org.junit.Test; + +public class TestACLConfigurationParser { + + @Test + public void testACLConfigParser() { + + Configuration conf = new Configuration(false); + String viewACLs = " user1,user4, grp3,grp4,grp5 "; + String modifyACLs = " user3 "; + conf.set(TezConfiguration.TEZ_AM_VIEW_ACLS, viewACLs); + + ACLConfigurationParser parser = new ACLConfigurationParser(conf); + Assert.assertTrue(parser.getAllowedUsers().containsKey(ACLType.AM_VIEW_ACL)); + Assert.assertFalse(parser.getAllowedUsers().containsKey(ACLType.AM_MODIFY_ACL)); + Assert.assertTrue(parser.getAllowedUsers().get(ACLType.AM_VIEW_ACL).contains("user1")); + Assert.assertFalse(parser.getAllowedUsers().get(ACLType.AM_VIEW_ACL).contains("user3")); + Assert.assertTrue(parser.getAllowedUsers().get(ACLType.AM_VIEW_ACL).contains("user4")); + Assert.assertFalse(parser.getAllowedGroups().isEmpty()); + Assert.assertTrue(parser.getAllowedGroups().get(ACLType.AM_VIEW_ACL).contains("grp3")); + Assert.assertFalse(parser.getAllowedGroups().get(ACLType.AM_VIEW_ACL).contains("grp6")); + Assert.assertTrue(parser.getAllowedGroups().get(ACLType.AM_VIEW_ACL).contains("grp4")); + Assert.assertTrue(parser.getAllowedGroups().get(ACLType.AM_VIEW_ACL).contains("grp5")); + + conf.set(TezConfiguration.TEZ_AM_MODIFY_ACLS, modifyACLs); + parser = new ACLConfigurationParser(conf); + Assert.assertTrue(parser.getAllowedUsers().containsKey(ACLType.AM_VIEW_ACL)); + Assert.assertTrue(parser.getAllowedUsers().containsKey(ACLType.AM_MODIFY_ACL)); + Assert.assertTrue(parser.getAllowedUsers().get(ACLType.AM_VIEW_ACL).contains("user1")); + Assert.assertFalse(parser.getAllowedUsers().get(ACLType.AM_VIEW_ACL).contains("user3")); + Assert.assertTrue(parser.getAllowedUsers().get(ACLType.AM_VIEW_ACL).contains("user4")); + Assert.assertFalse(parser.getAllowedUsers().get(ACLType.AM_MODIFY_ACL).contains("user1")); + Assert.assertTrue(parser.getAllowedUsers().get(ACLType.AM_MODIFY_ACL).contains("user3")); + Assert.assertFalse(parser.getAllowedGroups().isEmpty()); + Assert.assertTrue(parser.getAllowedGroups().get(ACLType.AM_VIEW_ACL).contains("grp3")); + Assert.assertFalse(parser.getAllowedGroups().get(ACLType.AM_VIEW_ACL).contains("grp6")); + Assert.assertTrue(parser.getAllowedGroups().get(ACLType.AM_VIEW_ACL).contains("grp4")); + Assert.assertTrue(parser.getAllowedGroups().get(ACLType.AM_VIEW_ACL).contains("grp5")); + Assert.assertNull(parser.getAllowedGroups().get(ACLType.AM_MODIFY_ACL)); + + } + +} diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/security/TestACLManager.java b/tez-dag/src/test/java/org/apache/tez/dag/app/security/TestACLManager.java new file mode 100644 index 0000000000..e413ed49b1 --- /dev/null +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/security/TestACLManager.java @@ -0,0 +1,368 @@ +/** + * 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.tez.dag.app.security; + +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; + +import java.io.IOException; +import java.util.Set; + +import org.apache.hadoop.conf.Configuration; +import org.apache.tez.dag.api.TezConfiguration; +import org.apache.tez.dag.app.security.ACLManager.ACLType; +import org.junit.Assert; +import org.junit.Test; + +import com.google.common.collect.Sets; + +public class TestACLManager { + + @Test + public void testCurrentUserACLChecks() { + Groups groups = null; + String currentUser = "currentUser"; + ACLManager aclManager = new ACLManager(groups, currentUser); + + String user = "user1"; + Assert.assertFalse(aclManager.checkAccess(user, ACLType.AM_VIEW_ACL)); + Assert.assertFalse(aclManager.checkAccess(user, ACLType.AM_MODIFY_ACL)); + + user = currentUser; + Assert.assertTrue(aclManager.checkAccess(user, ACLType.AM_VIEW_ACL)); + Assert.assertTrue(aclManager.checkAccess(user, ACLType.AM_MODIFY_ACL)); + + aclManager = new ACLManager(groups, currentUser, new Configuration(false)); + + user = "user1"; + Assert.assertFalse(aclManager.checkAccess(user, ACLType.AM_VIEW_ACL)); + Assert.assertFalse(aclManager.checkAccess(user, ACLType.AM_MODIFY_ACL)); + + user = currentUser; + Assert.assertTrue(aclManager.checkAccess(user, ACLType.AM_VIEW_ACL)); + Assert.assertTrue(aclManager.checkAccess(user, ACLType.AM_MODIFY_ACL)); + + String dagUser = "dagUser"; + ACLManager dagAclManager = new ACLManager(aclManager, dagUser, null); + user = dagUser; + Assert.assertFalse(dagAclManager.checkAccess(user, ACLType.AM_VIEW_ACL)); + Assert.assertFalse(dagAclManager.checkAccess(user, ACLType.AM_MODIFY_ACL)); + Assert.assertTrue(dagAclManager.checkAccess(user, ACLType.DAG_VIEW_ACL)); + Assert.assertTrue(dagAclManager.checkAccess(user, ACLType.DAG_MODIFY_ACL)); + user = "user1"; + Assert.assertFalse(dagAclManager.checkAccess(user, ACLType.DAG_VIEW_ACL)); + Assert.assertFalse(dagAclManager.checkAccess(user, ACLType.DAG_MODIFY_ACL)); + } + + @Test + public void testOtherUserACLChecks() throws IOException { + Groups groups = mock(Groups.class); + Set groups1 = Sets.newHashSet("grp1", "grp2"); + Set groups2 = Sets.newHashSet("grp3", "grp4"); + Set groups3 = Sets.newHashSet("grp5", "grp6"); + + String currentUser = "currentUser"; + String user1 = "user1"; // belongs to grp1 and grp2 + String user2 = "user2"; // belongs to grp3 and grp4 + String user3 = "user3"; + String user4 = "user4"; + String user5 = "user5"; // belongs to grp5 and grp6 + String user6 = "user6"; + + doReturn(groups1).when(groups).getGroups(eq(user1)); + doReturn(groups2).when(groups).getGroups(eq(user2)); + doReturn(Sets.newHashSet()).when(groups).getGroups(user3); + doReturn(Sets.newHashSet()).when(groups).getGroups(user4); + doReturn(groups3).when(groups).getGroups(eq(user5)); + doReturn(Sets.newHashSet()).when(groups).getGroups(user6); + + Configuration conf = new Configuration(false); + // View ACLs: user1, user4, grp3, grp4. + String viewACLs = user1 + "," + user4 + + " " + "grp3,grp4 "; + // Modify ACLs: user3, grp6, grp7 + String modifyACLs = " " + user3 + " " + "grp6,grp7"; + conf.set(TezConfiguration.TEZ_AM_VIEW_ACLS, viewACLs); + conf.set(TezConfiguration.TEZ_AM_MODIFY_ACLS, modifyACLs); + + ACLManager aclManager = new ACLManager(groups, currentUser, conf); + + Assert.assertTrue(aclManager.checkAccess(currentUser, ACLType.AM_VIEW_ACL)); + Assert.assertTrue(aclManager.checkAccess(user1, ACLType.AM_VIEW_ACL)); + Assert.assertTrue(aclManager.checkAccess(user2, ACLType.AM_VIEW_ACL)); + Assert.assertFalse(aclManager.checkAccess(user3, ACLType.AM_VIEW_ACL)); + Assert.assertTrue(aclManager.checkAccess(user4, ACLType.AM_VIEW_ACL)); + Assert.assertFalse(aclManager.checkAccess(user5, ACLType.AM_VIEW_ACL)); + Assert.assertFalse(aclManager.checkAccess(user6, ACLType.AM_VIEW_ACL)); + + Assert.assertTrue(aclManager.checkAccess(currentUser, ACLType.AM_MODIFY_ACL)); + Assert.assertFalse(aclManager.checkAccess(user1, ACLType.AM_MODIFY_ACL)); + Assert.assertFalse(aclManager.checkAccess(user2, ACLType.AM_MODIFY_ACL)); + Assert.assertTrue(aclManager.checkAccess(user3, ACLType.AM_MODIFY_ACL)); + Assert.assertFalse(aclManager.checkAccess(user4, ACLType.AM_MODIFY_ACL)); + Assert.assertTrue(aclManager.checkAccess(user5, ACLType.AM_MODIFY_ACL)); + Assert.assertFalse(aclManager.checkAccess(user6, ACLType.AM_MODIFY_ACL)); + } + + @Test + public void testNoGroupsACLChecks() throws IOException { + Groups groups = mock(Groups.class); + Set groups1 = Sets.newHashSet("grp1", "grp2"); + Set groups2 = Sets.newHashSet("grp3", "grp4"); + Set groups3 = Sets.newHashSet("grp5", "grp6"); + + String currentUser = "currentUser"; + String user1 = "user1"; // belongs to grp1 and grp2 + String user2 = "user2"; // belongs to grp3 and grp4 + String user3 = "user3"; + String user4 = "user4"; + String user5 = "user5"; // belongs to grp5 and grp6 + String user6 = "user6"; + + doReturn(groups1).when(groups).getGroups(eq(user1)); + doReturn(groups2).when(groups).getGroups(eq(user2)); + doReturn(Sets.newHashSet()).when(groups).getGroups(user3); + doReturn(Sets.newHashSet()).when(groups).getGroups(user4); + doReturn(groups3).when(groups).getGroups(eq(user5)); + doReturn(Sets.newHashSet()).when(groups).getGroups(user6); + + Configuration conf = new Configuration(false); + // View ACLs: user1, user4 + String viewACLs = " " + user1 + "," + user4 + " "; + // Modify ACLs: user3 + String modifyACLs = " " + user3 + " "; + conf.set(TezConfiguration.TEZ_AM_VIEW_ACLS, viewACLs); + conf.set(TezConfiguration.TEZ_AM_MODIFY_ACLS, modifyACLs); + + ACLManager aclManager = new ACLManager(groups, currentUser, conf); + Assert.assertTrue(aclManager.checkAccess(currentUser, ACLType.AM_VIEW_ACL)); + Assert.assertTrue(aclManager.checkAccess(user1, ACLType.AM_VIEW_ACL)); + Assert.assertFalse(aclManager.checkAccess(user2, ACLType.AM_VIEW_ACL)); + Assert.assertFalse(aclManager.checkAccess(user3, ACLType.AM_VIEW_ACL)); + Assert.assertTrue(aclManager.checkAccess(user4, ACLType.AM_VIEW_ACL)); + Assert.assertFalse(aclManager.checkAccess(user5, ACLType.AM_VIEW_ACL)); + Assert.assertFalse(aclManager.checkAccess(user6, ACLType.AM_VIEW_ACL)); + + Assert.assertTrue(aclManager.checkAccess(currentUser, ACLType.AM_MODIFY_ACL)); + Assert.assertFalse(aclManager.checkAccess(user1, ACLType.AM_MODIFY_ACL)); + Assert.assertFalse(aclManager.checkAccess(user2, ACLType.AM_MODIFY_ACL)); + Assert.assertTrue(aclManager.checkAccess(user3, ACLType.AM_MODIFY_ACL)); + Assert.assertFalse(aclManager.checkAccess(user4, ACLType.AM_MODIFY_ACL)); + Assert.assertFalse(aclManager.checkAccess(user5, ACLType.AM_MODIFY_ACL)); + Assert.assertFalse(aclManager.checkAccess(user6, ACLType.AM_MODIFY_ACL)); + } + + @Test + public void checkAMACLs() throws IOException { + Groups groups = mock(Groups.class); + Set groups1 = Sets.newHashSet("grp1", "grp2"); + Set groups2 = Sets.newHashSet("grp3", "grp4"); + Set groups3 = Sets.newHashSet("grp5", "grp6"); + + String currentUser = "currentUser"; + String user1 = "user1"; // belongs to grp1 and grp2 + String user2 = "user2"; // belongs to grp3 and grp4 + String user3 = "user3"; + String user4 = "user4"; + String user5 = "user5"; // belongs to grp5 and grp6 + String user6 = "user6"; + + doReturn(groups1).when(groups).getGroups(eq(user1)); + doReturn(groups2).when(groups).getGroups(eq(user2)); + doReturn(Sets.newHashSet()).when(groups).getGroups(user3); + doReturn(Sets.newHashSet()).when(groups).getGroups(user4); + doReturn(groups3).when(groups).getGroups(eq(user5)); + doReturn(Sets.newHashSet()).when(groups).getGroups(user6); + + Configuration conf = new Configuration(false); + // View ACLs: user1, user4, grp3, grp4. + String viewACLs = " user1,user4,, grp3,grp4 "; + // Modify ACLs: user3, grp6, grp7 + String modifyACLs = " user3 grp6,grp7"; + conf.set(TezConfiguration.TEZ_AM_VIEW_ACLS, viewACLs); + conf.set(TezConfiguration.TEZ_AM_MODIFY_ACLS, modifyACLs); + + ACLManager aclManager = new ACLManager(groups, currentUser, conf); + + Assert.assertTrue(aclManager.checkAMViewAccess(currentUser)); + Assert.assertTrue(aclManager.checkAMViewAccess(user1)); + Assert.assertTrue(aclManager.checkAMViewAccess(user2)); + Assert.assertFalse(aclManager.checkAMViewAccess(user3)); + Assert.assertTrue(aclManager.checkAMViewAccess(user4)); + Assert.assertFalse(aclManager.checkAMViewAccess(user5)); + Assert.assertFalse(aclManager.checkAMViewAccess(user6)); + + Assert.assertTrue(aclManager.checkAMModifyAccess(currentUser)); + Assert.assertFalse(aclManager.checkAMModifyAccess(user1)); + Assert.assertFalse(aclManager.checkAMModifyAccess(user2)); + Assert.assertTrue(aclManager.checkAMModifyAccess(user3)); + Assert.assertFalse(aclManager.checkAMModifyAccess(user4)); + Assert.assertTrue(aclManager.checkAMModifyAccess(user5)); + Assert.assertFalse(aclManager.checkAMModifyAccess(user6)); + + Assert.assertTrue(aclManager.checkDAGViewAccess(currentUser)); + Assert.assertTrue(aclManager.checkDAGViewAccess(user1)); + Assert.assertTrue(aclManager.checkDAGViewAccess(user2)); + Assert.assertFalse(aclManager.checkDAGViewAccess(user3)); + Assert.assertTrue(aclManager.checkDAGViewAccess(user4)); + Assert.assertFalse(aclManager.checkDAGViewAccess(user5)); + Assert.assertFalse(aclManager.checkDAGViewAccess(user6)); + + Assert.assertTrue(aclManager.checkDAGModifyAccess(currentUser)); + Assert.assertFalse(aclManager.checkDAGModifyAccess(user1)); + Assert.assertFalse(aclManager.checkDAGModifyAccess(user2)); + Assert.assertTrue(aclManager.checkDAGModifyAccess(user3)); + Assert.assertFalse(aclManager.checkDAGModifyAccess(user4)); + Assert.assertTrue(aclManager.checkDAGModifyAccess(user5)); + Assert.assertFalse(aclManager.checkDAGModifyAccess(user6)); + + } + + @Test + public void checkDAGACLs() throws IOException { + Groups groups = mock(Groups.class); + Set groups1 = Sets.newHashSet("grp1", "grp2"); + Set groups2 = Sets.newHashSet("grp3", "grp4"); + Set groups3 = Sets.newHashSet("grp5", "grp6"); + + String currentUser = "currentUser"; + String user1 = "user1"; // belongs to grp1 and grp2 + String user2 = "user2"; // belongs to grp3 and grp4 + String user3 = "user3"; + String user4 = "user4"; + String user5 = "user5"; // belongs to grp5 and grp6 + String user6 = "user6"; + + doReturn(groups1).when(groups).getGroups(eq(user1)); + doReturn(groups2).when(groups).getGroups(eq(user2)); + doReturn(Sets.newHashSet()).when(groups).getGroups(user3); + doReturn(Sets.newHashSet()).when(groups).getGroups(user4); + doReturn(groups3).when(groups).getGroups(eq(user5)); + doReturn(Sets.newHashSet()).when(groups).getGroups(user6); + + Configuration conf = new Configuration(false); + // View ACLs: user1, user4, grp3, grp4. + String viewACLs = " user1,user4,, grp3,grp4 "; + // Modify ACLs: user3, grp6, grp7 + String modifyACLs = " user3 grp6,grp7"; + conf.set(TezConfiguration.TEZ_AM_VIEW_ACLS, viewACLs); + conf.set(TezConfiguration.TEZ_AM_MODIFY_ACLS, modifyACLs); + + // DAG View ACLs: user1, user4, grp3, grp4. + String dagViewACLs = " user6, grp5 "; + // DAG Modify ACLs: user3, grp6, grp7 + String dagModifyACLs = " user6,user5 "; + conf.set(TezConfiguration.TEZ_DAG_VIEW_ACLS, dagViewACLs); + conf.set(TezConfiguration.TEZ_DAG_MODIFY_ACLS, dagModifyACLs); + + String dagUser = "dagUser"; + + ACLManager amAclManager = new ACLManager(groups, currentUser, conf); + ACLManager aclManager = new ACLManager(amAclManager, dagUser, conf); + + Assert.assertTrue(aclManager.checkAMViewAccess(currentUser)); + Assert.assertFalse(aclManager.checkAMViewAccess(dagUser)); + Assert.assertTrue(aclManager.checkAMViewAccess(user1)); + Assert.assertTrue(aclManager.checkAMViewAccess(user2)); + Assert.assertFalse(aclManager.checkAMViewAccess(user3)); + Assert.assertTrue(aclManager.checkAMViewAccess(user4)); + Assert.assertFalse(aclManager.checkAMViewAccess(user5)); + Assert.assertFalse(aclManager.checkAMViewAccess(user6)); + + Assert.assertTrue(aclManager.checkAMModifyAccess(currentUser)); + Assert.assertFalse(aclManager.checkAMModifyAccess(dagUser)); + Assert.assertFalse(aclManager.checkAMModifyAccess(user1)); + Assert.assertFalse(aclManager.checkAMModifyAccess(user2)); + Assert.assertTrue(aclManager.checkAMModifyAccess(user3)); + Assert.assertFalse(aclManager.checkAMModifyAccess(user4)); + Assert.assertTrue(aclManager.checkAMModifyAccess(user5)); + Assert.assertFalse(aclManager.checkAMModifyAccess(user6)); + + Assert.assertTrue(aclManager.checkDAGViewAccess(currentUser)); + Assert.assertTrue(aclManager.checkDAGViewAccess(dagUser)); + Assert.assertTrue(aclManager.checkDAGViewAccess(user1)); + Assert.assertTrue(aclManager.checkDAGViewAccess(user2)); + Assert.assertFalse(aclManager.checkDAGViewAccess(user3)); + Assert.assertTrue(aclManager.checkDAGViewAccess(user4)); + Assert.assertTrue(aclManager.checkDAGViewAccess(user5)); + Assert.assertTrue(aclManager.checkDAGViewAccess(user6)); + + Assert.assertTrue(aclManager.checkDAGModifyAccess(currentUser)); + Assert.assertTrue(aclManager.checkDAGModifyAccess(dagUser)); + Assert.assertFalse(aclManager.checkDAGModifyAccess(user1)); + Assert.assertFalse(aclManager.checkDAGModifyAccess(user2)); + Assert.assertTrue(aclManager.checkDAGModifyAccess(user3)); + Assert.assertFalse(aclManager.checkDAGModifyAccess(user4)); + Assert.assertTrue(aclManager.checkDAGModifyAccess(user5)); + Assert.assertTrue(aclManager.checkDAGModifyAccess(user6)); + + } + + @Test + public void testWildCardCheck() { + Groups groups = mock(Groups.class); + Configuration conf = new Configuration(false); + String viewACLs = " * "; + String modifyACLs = " * "; + conf.set(TezConfiguration.TEZ_AM_VIEW_ACLS, viewACLs); + conf.set(TezConfiguration.TEZ_AM_MODIFY_ACLS, modifyACLs); + + ACLManager aclManager = new ACLManager(groups, "a1", conf); + Assert.assertTrue(aclManager.checkAMViewAccess("a1")); + Assert.assertTrue(aclManager.checkAMViewAccess("u1")); + Assert.assertTrue(aclManager.checkAMModifyAccess("a1")); + Assert.assertTrue(aclManager.checkAMModifyAccess("u1")); + Assert.assertTrue(aclManager.checkDAGViewAccess("a1")); + Assert.assertTrue(aclManager.checkDAGViewAccess("u1")); + Assert.assertTrue(aclManager.checkDAGModifyAccess("a1")); + Assert.assertTrue(aclManager.checkDAGModifyAccess("u1")); + } + + @Test + public void testACLsDisabled() { + Groups groups = mock(Groups.class); + Configuration conf = new Configuration(false); + conf.setBoolean(TezConfiguration.TEZ_AM_ACLS_ENABLED, false); + String viewACLs = " a2,u2 "; + String modifyACLs = " a2,u2 "; + conf.set(TezConfiguration.TEZ_AM_VIEW_ACLS, viewACLs); + conf.set(TezConfiguration.TEZ_AM_MODIFY_ACLS, modifyACLs); + ACLManager aclManager = new ACLManager(groups, "a1", conf); + Assert.assertTrue(aclManager.checkAMViewAccess("a1")); + Assert.assertTrue(aclManager.checkAMViewAccess("u1")); + Assert.assertTrue(aclManager.checkAMModifyAccess("a1")); + Assert.assertTrue(aclManager.checkAMModifyAccess("u1")); + Assert.assertTrue(aclManager.checkDAGViewAccess("a1")); + Assert.assertTrue(aclManager.checkDAGViewAccess("u1")); + Assert.assertTrue(aclManager.checkDAGModifyAccess("a1")); + Assert.assertTrue(aclManager.checkDAGModifyAccess("u1")); + + ACLManager dagAclManager = new ACLManager(aclManager, "dagUser", null); + Assert.assertTrue(dagAclManager.checkAMViewAccess("a1")); + Assert.assertTrue(dagAclManager.checkAMViewAccess("u1")); + Assert.assertTrue(dagAclManager.checkAMModifyAccess("a1")); + Assert.assertTrue(dagAclManager.checkAMModifyAccess("u1")); + Assert.assertTrue(dagAclManager.checkDAGViewAccess("a1")); + Assert.assertTrue(dagAclManager.checkDAGViewAccess("u1")); + Assert.assertTrue(dagAclManager.checkDAGModifyAccess("a1")); + Assert.assertTrue(dagAclManager.checkDAGModifyAccess("u1")); + } + +} diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezMergedInputContextImpl.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezMergedInputContextImpl.java index cf55d39721..1e79dc3ff4 100644 --- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezMergedInputContextImpl.java +++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezMergedInputContextImpl.java @@ -1,3 +1,21 @@ +/** + * 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.tez.runtime.api.impl; import static com.google.common.base.Preconditions.checkNotNull; diff --git a/tez-tools/swimlanes/amlogparser.py b/tez-tools/swimlanes/amlogparser.py index 46d059987d..4556ab78ab 100644 --- a/tez-tools/swimlanes/amlogparser.py +++ b/tez-tools/swimlanes/amlogparser.py @@ -1,3 +1,22 @@ +# +# 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. +# + import os,sys,re,math,os.path from collections import defaultdict from itertools import groupby diff --git a/tez-tools/swimlanes/swimlane.py b/tez-tools/swimlanes/swimlane.py index 5a527ba09c..e08c648db7 100644 --- a/tez-tools/swimlanes/swimlane.py +++ b/tez-tools/swimlanes/swimlane.py @@ -1,3 +1,22 @@ +# +# 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. +# + import os,sys,re,math,os.path import StringIO from amlogparser import AMLog From dd5915cedaaa4da163b74a42667a06c0474a6bd2 Mon Sep 17 00:00:00 2001 From: Hitesh Shah Date: Thu, 7 Aug 2014 14:44:57 -0700 Subject: [PATCH 2/9] Address Oleg's comment on handling of RPC user. --- ...GClientAMProtocolBlockingPBServerImpl.java | 62 +++++-------------- 1 file changed, 15 insertions(+), 47 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java index 4e39fd3e33..a82f7b97f1 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java @@ -62,15 +62,18 @@ public DAGClientAMProtocolBlockingPBServerImpl(DAGClientHandler real) { this.real = real; } - @Override - public GetAllDAGsResponseProto getAllDAGs(RpcController controller, - GetAllDAGsRequestProto request) throws ServiceException { - String user; + private String getRPCUserName() throws ServiceException { try { - user = UserGroupInformation.getCurrentUser().getShortUserName(); + return UserGroupInformation.getCurrentUser().getShortUserName(); } catch (IOException e) { throw wrapException(e); } + } + + @Override + public GetAllDAGsResponseProto getAllDAGs(RpcController controller, + GetAllDAGsRequestProto request) throws ServiceException { + String user = getRPCUserName(); if (!real.getACLManager().checkAMViewAccess(user)) { throw new AccessControlException("User " + user + " cannot perform AM view operation"); } @@ -85,12 +88,7 @@ public GetAllDAGsResponseProto getAllDAGs(RpcController controller, @Override public GetDAGStatusResponseProto getDAGStatus(RpcController controller, GetDAGStatusRequestProto request) throws ServiceException { - String user; - try { - user = UserGroupInformation.getCurrentUser().getShortUserName(); - } catch (IOException e) { - throw wrapException(e); - } + String user = getRPCUserName(); try { String dagId = request.getDagId(); if (!real.getACLManager(dagId).checkDAGViewAccess(user)) { @@ -112,12 +110,7 @@ public GetDAGStatusResponseProto getDAGStatus(RpcController controller, @Override public GetVertexStatusResponseProto getVertexStatus(RpcController controller, GetVertexStatusRequestProto request) throws ServiceException { - String user; - try { - user = UserGroupInformation.getCurrentUser().getShortUserName(); - } catch (IOException e) { - throw wrapException(e); - } + String user = getRPCUserName(); try { String dagId = request.getDagId(); if (!real.getACLManager(dagId).checkDAGViewAccess(user)) { @@ -139,12 +132,7 @@ public GetVertexStatusResponseProto getVertexStatus(RpcController controller, @Override public TryKillDAGResponseProto tryKillDAG(RpcController controller, TryKillDAGRequestProto request) throws ServiceException { - String user; - try { - user = UserGroupInformation.getCurrentUser().getShortUserName(); - } catch (IOException e) { - throw wrapException(e); - } + String user = getRPCUserName(); try { String dagId = request.getDagId(); if (!real.getACLManager(dagId).checkDAGModifyAccess(user)) { @@ -160,12 +148,7 @@ public TryKillDAGResponseProto tryKillDAG(RpcController controller, @Override public SubmitDAGResponseProto submitDAG(RpcController controller, SubmitDAGRequestProto request) throws ServiceException { - String user; - try { - user = UserGroupInformation.getCurrentUser().getShortUserName(); - } catch (IOException e) { - throw wrapException(e); - } + String user = getRPCUserName(); if (!real.getACLManager().checkAMModifyAccess(user)) { throw new AccessControlException("User " + user + " cannot perform AM modify operation"); } @@ -190,12 +173,7 @@ ServiceException wrapException(Exception e){ @Override public ShutdownSessionResponseProto shutdownSession(RpcController arg0, ShutdownSessionRequestProto arg1) throws ServiceException { - String user; - try { - user = UserGroupInformation.getCurrentUser().getShortUserName(); - } catch (IOException e) { - throw wrapException(e); - } + String user = getRPCUserName(); if (!real.getACLManager().checkAMModifyAccess(user)) { throw new AccessControlException("User " + user + " cannot perform AM modify operation"); } @@ -206,12 +184,7 @@ public ShutdownSessionResponseProto shutdownSession(RpcController arg0, @Override public GetAMStatusResponseProto getAMStatus(RpcController controller, GetAMStatusRequestProto request) throws ServiceException { - String user; - try { - user = UserGroupInformation.getCurrentUser().getShortUserName(); - } catch (IOException e) { - throw wrapException(e); - } + String user = getRPCUserName(); if (!real.getACLManager().checkAMViewAccess(user)) { throw new AccessControlException("User " + user + " cannot perform AM view operation"); } @@ -229,12 +202,7 @@ public GetAMStatusResponseProto getAMStatus(RpcController controller, public DAGClientAMProtocolRPC.PreWarmResponseProto preWarm( RpcController controller, PreWarmRequestProto request) throws ServiceException { - String user; - try { - user = UserGroupInformation.getCurrentUser().getShortUserName(); - } catch (IOException e) { - throw wrapException(e); - } + String user = getRPCUserName(); if (!real.getACLManager().checkAMModifyAccess(user)) { throw new AccessControlException("User " + user + " cannot perform AM modify operation"); } From 815f718a2838e87f511266adb617a1f72033c963 Mon Sep 17 00:00:00 2001 From: Hitesh Shah Date: Thu, 7 Aug 2014 11:49:15 -0700 Subject: [PATCH 3/9] TEZ-671. Support View/Modify ACLs for DAGs. --- .../tez/dag/api/client/DAGClientHandler.java | 18 ++---------------- 1 file changed, 2 insertions(+), 16 deletions(-) diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGClientHandler.java b/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGClientHandler.java index d2d58f2696..0920deee1a 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGClientHandler.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGClientHandler.java @@ -135,22 +135,8 @@ public ACLManager getACLManager() { } public ACLManager getACLManager(String dagIdStr) throws TezException { - TezDAGID dagId = TezDAGID.fromString(dagIdStr); - if (dagId == null) { - throw new TezException("Bad dagId: " + dagIdStr); - } - DAG currentDAG = getCurrentDAG(); - if (currentDAG == null) { - throw new TezException("No running dag at present"); - } - if (!currentDAG.getID().toString().equals(dagId.toString())) { - LOG.warn("Current DAGID : " - + (currentDAG.getID() == null ? "NULL" : currentDAG.getID()) - + ", Looking for string (not found): " + dagIdStr + ", dagIdObj: " - + dagId); - throw new TezException("Unknown dagId: " + dagIdStr); - } - return currentDAG.getACLManager(); + DAG dag = getDAG(dagIdStr); + return dag.getACLManager(); } } From cdfca19e440f1637283178a9a91ce12ad6051e2c Mon Sep 17 00:00:00 2001 From: Hitesh Shah Date: Fri, 8 Aug 2014 18:33:20 -0700 Subject: [PATCH 4/9] WIP to address comments --- .../org/apache/tez/client/TezClientUtils.java | 17 +++++++-- .../security/ACLConfigurationParser.java | 4 +- .../tez/common}/security/ACLManager.java | 37 ++++++++++++++++++- .../apache/tez/common}/security/Groups.java | 2 +- .../security/TestACLConfigurationParser.java | 5 ++- .../tez/common}/security/TestACLManager.java | 6 ++- .../tez/dag/api/client/DAGClientHandler.java | 2 +- .../org/apache/tez/dag/app/AppContext.java | 2 +- .../org/apache/tez/dag/app/DAGAppMaster.java | 4 +- .../java/org/apache/tez/dag/app/dag/DAG.java | 3 +- .../apache/tez/dag/app/dag/impl/DAGImpl.java | 2 +- .../tez/dag/app/dag/impl/TestDAGImpl.java | 4 +- 12 files changed, 68 insertions(+), 20 deletions(-) rename {tez-dag/src/main/java/org/apache/tez/dag/app => tez-api/src/main/java/org/apache/tez/common}/security/ACLConfigurationParser.java (97%) rename {tez-dag/src/main/java/org/apache/tez/dag/app => tez-api/src/main/java/org/apache/tez/common}/security/ACLManager.java (81%) rename {tez-dag/src/main/java/org/apache/tez/dag/app => tez-api/src/main/java/org/apache/tez/common}/security/Groups.java (99%) rename {tez-dag/src/test/java/org/apache/tez/dag/app => tez-api/src/test/java/org/apache/tez/common}/security/TestACLConfigurationParser.java (95%) rename {tez-dag/src/test/java/org/apache/tez/dag/app => tez-api/src/test/java/org/apache/tez/common}/security/TestACLManager.java (98%) diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java index d05dd7e13a..ce2ad68a08 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java @@ -74,6 +74,8 @@ import org.apache.tez.common.TezCommonUtils; import org.apache.tez.common.TezYARNUtils; import org.apache.tez.common.impl.LogUtils; +import org.apache.tez.common.security.ACLManager; +import org.apache.tez.common.security.Groups; import org.apache.tez.common.security.JobTokenIdentifier; import org.apache.tez.common.security.JobTokenSecretManager; import org.apache.tez.common.security.TokenCache; @@ -553,6 +555,14 @@ static ApplicationSubmissionContext createApplicationSubmissionContext( TezConfiguration.TEZ_SESSION_LOCAL_RESOURCES_PB_FILE_NAME, sessionJarsPBLRsrc); + String user = UserGroupInformation.getCurrentUser().getShortUserName(); + + Groups groups = null; + ACLManager aclManager = new ACLManager(groups, user, finalTezConf); + + Map acls = aclManager.toYARNACls(); + + if(dag != null) { for (Vertex v : dag.getVertices()) { @@ -607,8 +617,9 @@ static ApplicationSubmissionContext createApplicationSubmissionContext( LocalResourceVisibility.APPLICATION)); } } - Map acls - = new HashMap(); + + + // Setup ContainerLaunchContext for AM container ContainerLaunchContext amContainer = @@ -630,7 +641,7 @@ static ApplicationSubmissionContext createApplicationSubmissionContext( TezConfiguration.TEZ_AM_CANCEL_DELEGATION_TOKEN, TezConfiguration.TEZ_AM_CANCEL_DELEGATION_TOKEN_DEFAULT)); appContext.setAMContainerSpec(amContainer); - + appContext.setMaxAppAttempts( finalTezConf.getInt(TezConfiguration.TEZ_AM_MAX_APP_ATTEMPTS, TezConfiguration.TEZ_AM_MAX_APP_ATTEMPTS_DEFAULT)); diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/security/ACLConfigurationParser.java b/tez-api/src/main/java/org/apache/tez/common/security/ACLConfigurationParser.java similarity index 97% rename from tez-dag/src/main/java/org/apache/tez/dag/app/security/ACLConfigurationParser.java rename to tez-api/src/main/java/org/apache/tez/common/security/ACLConfigurationParser.java index c72561e9ac..4f966bc996 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/security/ACLConfigurationParser.java +++ b/tez-api/src/main/java/org/apache/tez/common/security/ACLConfigurationParser.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.tez.dag.app.security; +package org.apache.tez.common.security; import java.util.Collections; import java.util.HashMap; @@ -28,8 +28,8 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.util.StringUtils; +import org.apache.tez.common.security.ACLManager.ACLType; import org.apache.tez.dag.api.TezConfiguration; -import org.apache.tez.dag.app.security.ACLManager.ACLType; import com.google.common.collect.Sets; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/security/ACLManager.java b/tez-api/src/main/java/org/apache/tez/common/security/ACLManager.java similarity index 81% rename from tez-dag/src/main/java/org/apache/tez/dag/app/security/ACLManager.java rename to tez-api/src/main/java/org/apache/tez/common/security/ACLManager.java index ec3b51cd4f..ba06c6a88f 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/security/ACLManager.java +++ b/tez-api/src/main/java/org/apache/tez/common/security/ACLManager.java @@ -16,17 +16,19 @@ * limitations under the License. */ -package org.apache.tez.dag.app.security; +package org.apache.tez.common.security; import java.io.IOException; import java.util.EnumSet; import java.util.HashMap; import java.util.Map; +import java.util.Map.Entry; import java.util.Set; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.api.records.ApplicationAccessType; import org.apache.tez.dag.api.TezConfiguration; import com.google.common.annotations.VisibleForTesting; @@ -164,6 +166,39 @@ public boolean checkDAGModifyAccess(String user) { || checkAccess(user, ACLType.DAG_MODIFY_ACL); } + public Map toYARNACls() { + Map acls = new HashMap(2); + if (!this.aclsEnabled) { + acls.put(ApplicationAccessType.VIEW_APP, "*"); + acls.put(ApplicationAccessType.MODIFY_APP, "*"); + return acls; + } + boolean viewAclsWildCard = false; + boolean modifyAclsWildCard = false; + if (users != null && !users.isEmpty()) { + for (Entry> entry : users.entrySet()) { + if (entry.getValue().contains(WILDCARD_ACL_VALUE)) { + if (entry.getKey().equals(ACLType.AM_VIEW_ACL)) { + viewAclsWildCard = true; + acls.put(ApplicationAccessType.VIEW_APP, "*"); + continue; + } + } else if (!entry.getValue().isEmpty()) { + + + } + } + } + if (groups != null && !groups.isEmpty()) { + for (Entry> entry : groups.entrySet()) { + if (viewAclsWildCard && + (entry.getKey().equals(ACLType.AM_VIEW_ACL)) + + } + } + return acls; + } + } diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/security/Groups.java b/tez-api/src/main/java/org/apache/tez/common/security/Groups.java similarity index 99% rename from tez-dag/src/main/java/org/apache/tez/dag/app/security/Groups.java rename to tez-api/src/main/java/org/apache/tez/common/security/Groups.java index 624156c0d6..ce6f1dd5d9 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/security/Groups.java +++ b/tez-api/src/main/java/org/apache/tez/common/security/Groups.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.tez.dag.app.security; +package org.apache.tez.common.security; import java.io.IOException; import java.util.Collection; diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/security/TestACLConfigurationParser.java b/tez-api/src/test/java/org/apache/tez/common/security/TestACLConfigurationParser.java similarity index 95% rename from tez-dag/src/test/java/org/apache/tez/dag/app/security/TestACLConfigurationParser.java rename to tez-api/src/test/java/org/apache/tez/common/security/TestACLConfigurationParser.java index 1070d606ce..df4304dd44 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/security/TestACLConfigurationParser.java +++ b/tez-api/src/test/java/org/apache/tez/common/security/TestACLConfigurationParser.java @@ -16,11 +16,12 @@ * limitations under the License. */ -package org.apache.tez.dag.app.security; +package org.apache.tez.common.security; import org.apache.hadoop.conf.Configuration; +import org.apache.tez.common.security.ACLConfigurationParser; import org.apache.tez.dag.api.TezConfiguration; -import org.apache.tez.dag.app.security.ACLManager.ACLType; +import org.apache.tez.common.security.ACLManager.ACLType; import org.junit.Assert; import org.junit.Test; diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/security/TestACLManager.java b/tez-api/src/test/java/org/apache/tez/common/security/TestACLManager.java similarity index 98% rename from tez-dag/src/test/java/org/apache/tez/dag/app/security/TestACLManager.java rename to tez-api/src/test/java/org/apache/tez/common/security/TestACLManager.java index e413ed49b1..04eea71d07 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/security/TestACLManager.java +++ b/tez-api/src/test/java/org/apache/tez/common/security/TestACLManager.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.tez.dag.app.security; +package org.apache.tez.common.security; import static org.mockito.Matchers.eq; import static org.mockito.Mockito.doReturn; @@ -26,8 +26,10 @@ import java.util.Set; import org.apache.hadoop.conf.Configuration; +import org.apache.tez.common.security.ACLManager; +import org.apache.tez.common.security.Groups; import org.apache.tez.dag.api.TezConfiguration; -import org.apache.tez.dag.app.security.ACLManager.ACLType; +import org.apache.tez.common.security.ACLManager.ACLType; import org.junit.Assert; import org.junit.Test; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGClientHandler.java b/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGClientHandler.java index 0920deee1a..c4d3497dc3 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGClientHandler.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGClientHandler.java @@ -31,7 +31,7 @@ import org.apache.tez.dag.api.records.DAGProtos.DAGPlan; import org.apache.tez.dag.app.DAGAppMaster; import org.apache.tez.dag.app.dag.DAG; -import org.apache.tez.dag.app.security.ACLManager; +import org.apache.tez.common.security.ACLManager; import org.apache.tez.dag.records.TezDAGID; public class DAGClientHandler { diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java b/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java index fc76cdc947..89c468b569 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java @@ -33,7 +33,7 @@ import org.apache.tez.dag.app.rm.TaskSchedulerEventHandler; import org.apache.tez.dag.app.rm.container.AMContainerMap; import org.apache.tez.dag.app.rm.node.AMNodeMap; -import org.apache.tez.dag.app.security.ACLManager; +import org.apache.tez.common.security.ACLManager; import org.apache.tez.dag.history.HistoryEventHandler; import org.apache.tez.dag.records.TezDAGID; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java index 8436148a11..3f39e345d0 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java @@ -140,8 +140,8 @@ import org.apache.tez.dag.app.rm.container.ContainerSignatureMatcher; import org.apache.tez.dag.app.rm.node.AMNodeEventType; import org.apache.tez.dag.app.rm.node.AMNodeMap; -import org.apache.tez.dag.app.security.ACLManager; -import org.apache.tez.dag.app.security.Groups; +import org.apache.tez.common.security.ACLManager; +import org.apache.tez.common.security.Groups; import org.apache.tez.dag.history.DAGHistoryEvent; import org.apache.tez.dag.history.HistoryEventHandler; import org.apache.tez.dag.history.events.AMLaunchedEvent; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java index 4ef19966e1..2fae860c05 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java @@ -25,13 +25,12 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.Credentials; import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.yarn.api.records.ApplicationAccessType; import org.apache.tez.common.counters.TezCounters; import org.apache.tez.dag.api.client.DAGStatusBuilder; import org.apache.tez.dag.api.client.StatusGetOpts; import org.apache.tez.dag.api.client.VertexStatusBuilder; import org.apache.tez.dag.api.records.DAGProtos.DAGPlan; -import org.apache.tez.dag.app.security.ACLManager; +import org.apache.tez.common.security.ACLManager; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.records.TezDAGID; import org.apache.tez.dag.records.TezVertexID; diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java index 91eb556a44..84c1f3ab85 100644 --- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java +++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java @@ -93,7 +93,7 @@ import org.apache.tez.dag.app.dag.event.VertexEventRecoverVertex; import org.apache.tez.dag.app.dag.event.VertexEventTermination; import org.apache.tez.dag.app.dag.event.VertexEventType; -import org.apache.tez.dag.app.security.ACLManager; +import org.apache.tez.common.security.ACLManager; import org.apache.tez.dag.history.DAGHistoryEvent; import org.apache.tez.dag.history.HistoryEvent; import org.apache.tez.dag.history.events.DAGCommitStartedEvent; diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java index f8454624f1..574b9395f1 100644 --- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java +++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java @@ -91,8 +91,8 @@ import org.apache.tez.dag.app.dag.event.VertexEventTaskReschedule; import org.apache.tez.dag.app.dag.event.VertexEventType; import org.apache.tez.dag.app.dag.impl.TestVertexImpl.CountingOutputCommitter; -import org.apache.tez.dag.app.security.ACLManager; -import org.apache.tez.dag.app.security.Groups; +import org.apache.tez.common.security.ACLManager; +import org.apache.tez.common.security.Groups; import org.apache.tez.dag.history.HistoryEventHandler; import org.apache.tez.dag.records.TezDAGID; import org.apache.tez.dag.records.TezTaskID; From 8717ef80fdc85abcf1f805bdb8ed64bd91462c6a Mon Sep 17 00:00:00 2001 From: Hitesh Shah Date: Tue, 12 Aug 2014 10:23:19 -0700 Subject: [PATCH 5/9] Checkpoint yarn acls --- .../security/ACLConfigurationParser.java | 31 +++++---- .../tez/common/security/ACLManager.java | 63 ++++++++++++++++--- .../security/TestACLConfigurationParser.java | 19 +++++- .../tez/common/security/TestACLManager.java | 51 +++++++++++---- 4 files changed, 130 insertions(+), 34 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/common/security/ACLConfigurationParser.java b/tez-api/src/main/java/org/apache/tez/common/security/ACLConfigurationParser.java index 4f966bc996..50ba26cedf 100644 --- a/tez-api/src/main/java/org/apache/tez/common/security/ACLConfigurationParser.java +++ b/tez-api/src/main/java/org/apache/tez/common/security/ACLConfigurationParser.java @@ -64,18 +64,29 @@ private void parse(boolean dagACLs) { } } + private boolean isWildCard(String aclStr) { + return aclStr.trim().equals(ACLManager.WILDCARD_ACL_VALUE); + } + private void parseACLType(String configProperty, ACLType aclType) { String aclsStr = conf.get(configProperty); if (aclsStr == null || aclsStr.isEmpty()) { return; } + if (isWildCard(aclsStr)) { + allowedUsers.put(aclType, Sets.newHashSet(ACLManager.WILDCARD_ACL_VALUE)); + return; + } + final String[] splits = splitPattern.split(aclsStr); int counter = -1; String userListStr = null; String groupListStr = null; for (String s : splits) { if (s.isEmpty()) { - continue; + if (userListStr != null) { + continue; + } } ++counter; if (counter == 0) { @@ -91,17 +102,13 @@ private void parseACLType(String configProperty, ACLType aclType) { if (userListStr == null) { return; } - if (userListStr.equals(ACLManager.WILDCARD_ACL_VALUE)) { - allowedUsers.put(aclType, Sets.newHashSet(ACLManager.WILDCARD_ACL_VALUE)); - } else { - if (userListStr.length() >= 1) { - allowedUsers.put(aclType, - Sets.newHashSet(StringUtils.getTrimmedStringCollection(userListStr))); - } - if (groupListStr != null && groupListStr.length() >= 1) { - allowedGroups.put(aclType, - Sets.newHashSet(StringUtils.getTrimmedStringCollection(groupListStr))); - } + if (userListStr.length() >= 1) { + allowedUsers.put(aclType, + Sets.newHashSet(StringUtils.getTrimmedStringCollection(userListStr))); + } + if (groupListStr != null && groupListStr.length() >= 1) { + allowedGroups.put(aclType, + Sets.newHashSet(StringUtils.getTrimmedStringCollection(groupListStr))); } } diff --git a/tez-api/src/main/java/org/apache/tez/common/security/ACLManager.java b/tez-api/src/main/java/org/apache/tez/common/security/ACLManager.java index ba06c6a88f..41dcaa551d 100644 --- a/tez-api/src/main/java/org/apache/tez/common/security/ACLManager.java +++ b/tez-api/src/main/java/org/apache/tez/common/security/ACLManager.java @@ -19,6 +19,7 @@ package org.apache.tez.common.security; import java.io.IOException; +import java.util.Collection; import java.util.EnumSet; import java.util.HashMap; import java.util.Map; @@ -173,32 +174,76 @@ public Map toYARNACls() { acls.put(ApplicationAccessType.MODIFY_APP, "*"); return acls; } + acls.put(ApplicationAccessType.VIEW_APP, amUser); + acls.put(ApplicationAccessType.MODIFY_APP, amUser); boolean viewAclsWildCard = false; boolean modifyAclsWildCard = false; if (users != null && !users.isEmpty()) { for (Entry> entry : users.entrySet()) { - if (entry.getValue().contains(WILDCARD_ACL_VALUE)) { - if (entry.getKey().equals(ACLType.AM_VIEW_ACL)) { - viewAclsWildCard = true; + if (entry.getKey().equals(ACLType.AM_VIEW_ACL)) { + if (entry.getValue().contains(WILDCARD_ACL_VALUE)) { acls.put(ApplicationAccessType.VIEW_APP, "*"); + viewAclsWildCard = true; continue; + } else if (!entry.getValue().isEmpty()) { + String aclsStr = acls.get(ApplicationAccessType.VIEW_APP); + String commaSepList = toCommaSeparatedString(entry.getValue()); + if (!commaSepList.isEmpty()) { + aclsStr += "," + commaSepList; + } + acls.put(ApplicationAccessType.VIEW_APP, aclsStr); + } + } else if (entry.getKey().equals(ACLType.AM_MODIFY_ACL)) { + if (entry.getValue().contains(WILDCARD_ACL_VALUE)) { + acls.put(ApplicationAccessType.MODIFY_APP, "*"); + modifyAclsWildCard = true; + continue; + } else if (!entry.getValue().isEmpty()) { + String aclsStr = acls.get(ApplicationAccessType.MODIFY_APP); + String commaSepList = toCommaSeparatedString(entry.getValue()); + if (!commaSepList.isEmpty()) { + aclsStr += "," + commaSepList; + } + acls.put(ApplicationAccessType.MODIFY_APP, aclsStr); } - } else if (!entry.getValue().isEmpty()) { - - } } } if (groups != null && !groups.isEmpty()) { for (Entry> entry : groups.entrySet()) { - if (viewAclsWildCard && - (entry.getKey().equals(ACLType.AM_VIEW_ACL)) - + if (entry.getKey().equals(ACLType.AM_VIEW_ACL) + && !viewAclsWildCard && !entry.getValue().isEmpty()) { + // Append groups only if wild card not set + String aclsStr = acls.containsKey(ApplicationAccessType.VIEW_APP) ? + acls.get(ApplicationAccessType.VIEW_APP) : ""; + aclsStr += " " + toCommaSeparatedString(entry.getValue()); + acls.put(ApplicationAccessType.VIEW_APP, aclsStr); + } else if (entry.getKey().equals(ACLType.AM_MODIFY_ACL) + && !modifyAclsWildCard && !entry.getValue().isEmpty()) { + // Append groups only if wild card not set + String aclsStr = acls.containsKey(ApplicationAccessType.MODIFY_APP) ? + acls.get(ApplicationAccessType.MODIFY_APP) : ""; + aclsStr += " " + toCommaSeparatedString(entry.getValue()); + acls.put(ApplicationAccessType.MODIFY_APP, aclsStr); + } } } return acls; } + private String toCommaSeparatedString(Collection collection) { + StringBuilder sb = new StringBuilder(); + boolean first = true; + for (String s : collection) { + if (!first) { + sb.append(","); + } else { + first = false; + } + sb.append(s); + } + return sb.toString(); + } } diff --git a/tez-api/src/test/java/org/apache/tez/common/security/TestACLConfigurationParser.java b/tez-api/src/test/java/org/apache/tez/common/security/TestACLConfigurationParser.java index df4304dd44..6883e12fd4 100644 --- a/tez-api/src/test/java/org/apache/tez/common/security/TestACLConfigurationParser.java +++ b/tez-api/src/test/java/org/apache/tez/common/security/TestACLConfigurationParser.java @@ -19,6 +19,7 @@ package org.apache.tez.common.security; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.authorize.AccessControlList; import org.apache.tez.common.security.ACLConfigurationParser; import org.apache.tez.dag.api.TezConfiguration; import org.apache.tez.common.security.ACLManager.ACLType; @@ -31,8 +32,8 @@ public class TestACLConfigurationParser { public void testACLConfigParser() { Configuration conf = new Configuration(false); - String viewACLs = " user1,user4, grp3,grp4,grp5 "; - String modifyACLs = " user3 "; + String viewACLs = "user1,user4, grp3,grp4,grp5 "; + String modifyACLs = "user3 "; conf.set(TezConfiguration.TEZ_AM_VIEW_ACLS, viewACLs); ACLConfigurationParser parser = new ACLConfigurationParser(conf); @@ -65,4 +66,18 @@ public void testACLConfigParser() { } + @Test + public void testGroupsOnly() { + Configuration conf = new Configuration(false); + String viewACLs = " grp3,grp4,grp5"; + conf.set(TezConfiguration.TEZ_AM_VIEW_ACLS, viewACLs); + ACLConfigurationParser parser = new ACLConfigurationParser(conf); + Assert.assertTrue(parser.getAllowedUsers().isEmpty()); + Assert.assertFalse(parser.getAllowedGroups().isEmpty()); + Assert.assertTrue(parser.getAllowedGroups().get(ACLType.AM_VIEW_ACL).contains("grp3")); + Assert.assertFalse(parser.getAllowedGroups().get(ACLType.AM_VIEW_ACL).contains("grp6")); + Assert.assertTrue(parser.getAllowedGroups().get(ACLType.AM_VIEW_ACL).contains("grp4")); + Assert.assertTrue(parser.getAllowedGroups().get(ACLType.AM_VIEW_ACL).contains("grp5")); + } + } diff --git a/tez-api/src/test/java/org/apache/tez/common/security/TestACLManager.java b/tez-api/src/test/java/org/apache/tez/common/security/TestACLManager.java index 04eea71d07..aa8c979319 100644 --- a/tez-api/src/test/java/org/apache/tez/common/security/TestACLManager.java +++ b/tez-api/src/test/java/org/apache/tez/common/security/TestACLManager.java @@ -23,9 +23,11 @@ import static org.mockito.Mockito.mock; import java.io.IOException; +import java.util.Map; import java.util.Set; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.api.records.ApplicationAccessType; import org.apache.tez.common.security.ACLManager; import org.apache.tez.common.security.Groups; import org.apache.tez.dag.api.TezConfiguration; @@ -100,7 +102,7 @@ public void testOtherUserACLChecks() throws IOException { String viewACLs = user1 + "," + user4 + " " + "grp3,grp4 "; // Modify ACLs: user3, grp6, grp7 - String modifyACLs = " " + user3 + " " + "grp6,grp7"; + String modifyACLs = user3 + " " + "grp6,grp7"; conf.set(TezConfiguration.TEZ_AM_VIEW_ACLS, viewACLs); conf.set(TezConfiguration.TEZ_AM_MODIFY_ACLS, modifyACLs); @@ -147,9 +149,9 @@ public void testNoGroupsACLChecks() throws IOException { Configuration conf = new Configuration(false); // View ACLs: user1, user4 - String viewACLs = " " + user1 + "," + user4 + " "; + String viewACLs = user1 + "," + user4 + " "; // Modify ACLs: user3 - String modifyACLs = " " + user3 + " "; + String modifyACLs = "user3 "; conf.set(TezConfiguration.TEZ_AM_VIEW_ACLS, viewACLs); conf.set(TezConfiguration.TEZ_AM_MODIFY_ACLS, modifyACLs); @@ -195,9 +197,9 @@ public void checkAMACLs() throws IOException { Configuration conf = new Configuration(false); // View ACLs: user1, user4, grp3, grp4. - String viewACLs = " user1,user4,, grp3,grp4 "; + String viewACLs = "user1,user4,, grp3,grp4 "; // Modify ACLs: user3, grp6, grp7 - String modifyACLs = " user3 grp6,grp7"; + String modifyACLs = "user3 grp6,grp7"; conf.set(TezConfiguration.TEZ_AM_VIEW_ACLS, viewACLs); conf.set(TezConfiguration.TEZ_AM_MODIFY_ACLS, modifyACLs); @@ -261,16 +263,16 @@ public void checkDAGACLs() throws IOException { Configuration conf = new Configuration(false); // View ACLs: user1, user4, grp3, grp4. - String viewACLs = " user1,user4,, grp3,grp4 "; + String viewACLs = "user1,user4,, grp3,grp4 "; // Modify ACLs: user3, grp6, grp7 - String modifyACLs = " user3 grp6,grp7"; + String modifyACLs = "user3 grp6,grp7"; conf.set(TezConfiguration.TEZ_AM_VIEW_ACLS, viewACLs); conf.set(TezConfiguration.TEZ_AM_MODIFY_ACLS, modifyACLs); // DAG View ACLs: user1, user4, grp3, grp4. - String dagViewACLs = " user6, grp5 "; + String dagViewACLs = "user6, grp5 "; // DAG Modify ACLs: user3, grp6, grp7 - String dagModifyACLs = " user6,user5 "; + String dagModifyACLs = "user6,user5 "; conf.set(TezConfiguration.TEZ_DAG_VIEW_ACLS, dagViewACLs); conf.set(TezConfiguration.TEZ_DAG_MODIFY_ACLS, dagModifyACLs); @@ -342,8 +344,8 @@ public void testACLsDisabled() { Groups groups = mock(Groups.class); Configuration conf = new Configuration(false); conf.setBoolean(TezConfiguration.TEZ_AM_ACLS_ENABLED, false); - String viewACLs = " a2,u2 "; - String modifyACLs = " a2,u2 "; + String viewACLs = "a2,u2 "; + String modifyACLs = "a2,u2 "; conf.set(TezConfiguration.TEZ_AM_VIEW_ACLS, viewACLs); conf.set(TezConfiguration.TEZ_AM_MODIFY_ACLS, modifyACLs); ACLManager aclManager = new ACLManager(groups, "a1", conf); @@ -367,4 +369,31 @@ public void testACLsDisabled() { Assert.assertTrue(dagAclManager.checkDAGModifyAccess("u1")); } + @Test + public void testConvertToYARNACLs() { + Groups groups = mock(Groups.class); + String currentUser = "c1"; + Configuration conf = new Configuration(false); + String viewACLs = "user1,user4,, grp3,grp4 "; + conf.set(TezConfiguration.TEZ_AM_VIEW_ACLS, viewACLs); + conf.set(TezConfiguration.TEZ_AM_MODIFY_ACLS, " * "); + ACLManager aclManager = new ACLManager(groups, currentUser, conf); + + Map yarnAcls = aclManager.toYARNACls(); + Assert.assertTrue(yarnAcls.containsKey(ApplicationAccessType.VIEW_APP)); + Assert.assertEquals("c1,user1,user4 grp3,grp4", + yarnAcls.get(ApplicationAccessType.VIEW_APP)); + Assert.assertTrue(yarnAcls.containsKey(ApplicationAccessType.MODIFY_APP)); + Assert.assertEquals("*", + yarnAcls.get(ApplicationAccessType.MODIFY_APP)); + + viewACLs = " grp3,grp4 "; + conf.set(TezConfiguration.TEZ_AM_VIEW_ACLS, viewACLs); + ACLManager aclManager1 = new ACLManager(groups, currentUser, conf); + yarnAcls = aclManager1.toYARNACls(); + Assert.assertEquals("c1 grp3,grp4", + yarnAcls.get(ApplicationAccessType.VIEW_APP)); + + } + } From b9eefe6c5292bfc9eed79a60b2957794c96ad4c3 Mon Sep 17 00:00:00 2001 From: Hitesh Shah Date: Tue, 12 Aug 2014 12:55:59 -0700 Subject: [PATCH 6/9] whitespace cleanup --- .../src/main/java/org/apache/tez/client/TezClientUtils.java | 5 ----- 1 file changed, 5 deletions(-) diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java index ce2ad68a08..95e68ff9a5 100644 --- a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java +++ b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java @@ -559,10 +559,8 @@ static ApplicationSubmissionContext createApplicationSubmissionContext( Groups groups = null; ACLManager aclManager = new ACLManager(groups, user, finalTezConf); - Map acls = aclManager.toYARNACls(); - if(dag != null) { for (Vertex v : dag.getVertices()) { @@ -618,9 +616,6 @@ static ApplicationSubmissionContext createApplicationSubmissionContext( } } - - - // Setup ContainerLaunchContext for AM container ContainerLaunchContext amContainer = ContainerLaunchContext.newInstance(localResources, environment, From 7ed28638d3a9e31b51ed1d670bc2d05f664451db Mon Sep 17 00:00:00 2001 From: Hitesh Shah Date: Wed, 13 Aug 2014 13:58:12 -0700 Subject: [PATCH 7/9] Add DAGAccessControls, tests pending --- .../security/ACLConfigurationParser.java | 6 +- .../tez/common/security/ACLManager.java | 11 +- .../apache/tez/common/security/ACLType.java | 36 +++++ .../common/security/DAGAccessControls.java | 133 ++++++++++++++++++ .../main/java/org/apache/tez/dag/api/DAG.java | 12 ++ .../security/TestACLConfigurationParser.java | 3 - .../tez/common/security/TestACLManager.java | 3 - 7 files changed, 189 insertions(+), 15 deletions(-) create mode 100644 tez-api/src/main/java/org/apache/tez/common/security/ACLType.java create mode 100644 tez-api/src/main/java/org/apache/tez/common/security/DAGAccessControls.java diff --git a/tez-api/src/main/java/org/apache/tez/common/security/ACLConfigurationParser.java b/tez-api/src/main/java/org/apache/tez/common/security/ACLConfigurationParser.java index 50ba26cedf..4ec2e50caf 100644 --- a/tez-api/src/main/java/org/apache/tez/common/security/ACLConfigurationParser.java +++ b/tez-api/src/main/java/org/apache/tez/common/security/ACLConfigurationParser.java @@ -26,13 +26,17 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.util.StringUtils; -import org.apache.tez.common.security.ACLManager.ACLType; import org.apache.tez.dag.api.TezConfiguration; import com.google.common.collect.Sets; +/** + * Parser for extracting ACL information from Configs + */ +@Private public class ACLConfigurationParser { private static final Log LOG = LogFactory.getLog(ACLConfigurationParser.class); diff --git a/tez-api/src/main/java/org/apache/tez/common/security/ACLManager.java b/tez-api/src/main/java/org/apache/tez/common/security/ACLManager.java index 41dcaa551d..d4ae0537f9 100644 --- a/tez-api/src/main/java/org/apache/tez/common/security/ACLManager.java +++ b/tez-api/src/main/java/org/apache/tez/common/security/ACLManager.java @@ -28,6 +28,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.api.records.ApplicationAccessType; import org.apache.tez.dag.api.TezConfiguration; @@ -38,18 +39,12 @@ * Class to manage ACLs for the Tez AM and DAGs and provides functionality to check whether * a user is authorized to take certain actions. */ +@Private public class ACLManager { private static final Log LOG = LogFactory.getLog(ACLManager.class); static final String WILDCARD_ACL_VALUE = "*"; - static enum ACLType { - AM_VIEW_ACL, - AM_MODIFY_ACL, - DAG_VIEW_ACL, - DAG_MODIFY_ACL - } - private final String dagUser; private final String amUser; private final Map> users; @@ -231,7 +226,7 @@ public Map toYARNACls() { return acls; } - private String toCommaSeparatedString(Collection collection) { + static String toCommaSeparatedString(Collection collection) { StringBuilder sb = new StringBuilder(); boolean first = true; for (String s : collection) { diff --git a/tez-api/src/main/java/org/apache/tez/common/security/ACLType.java b/tez-api/src/main/java/org/apache/tez/common/security/ACLType.java new file mode 100644 index 0000000000..854e928db5 --- /dev/null +++ b/tez-api/src/main/java/org/apache/tez/common/security/ACLType.java @@ -0,0 +1,36 @@ +/** + * 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.tez.common.security; + +import org.apache.hadoop.classification.InterfaceAudience.Private; + +/** + * ACL Types + */ +@Private +enum ACLType { + /** View permissions on the Application Master */ + AM_VIEW_ACL, + /** Modify permissions on the Application Master */ + AM_MODIFY_ACL, + /** View permissions on the DAG */ + DAG_VIEW_ACL, + /** Modify permissions on the DAG */ + DAG_MODIFY_ACL +} diff --git a/tez-api/src/main/java/org/apache/tez/common/security/DAGAccessControls.java b/tez-api/src/main/java/org/apache/tez/common/security/DAGAccessControls.java new file mode 100644 index 0000000000..751bbd2387 --- /dev/null +++ b/tez-api/src/main/java/org/apache/tez/common/security/DAGAccessControls.java @@ -0,0 +1,133 @@ +/** + * 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.tez.common.security; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; + +import org.apache.hadoop.classification.InterfaceAudience.Private; +import org.apache.hadoop.conf.Configuration; +import org.apache.tez.dag.api.TezConfiguration; + +/** + * Access controls for the DAG + */ +public class DAGAccessControls { + + private final Set usersWithViewACLs; + private final Set usersWithModifyACLs; + private final Set groupsWithViewACLs; + private final Set groupsWithModifyACLs; + + public DAGAccessControls() { + this.usersWithViewACLs = new HashSet(); + this.usersWithModifyACLs = new HashSet(); + this.groupsWithViewACLs = new HashSet(); + this.groupsWithModifyACLs = new HashSet(); + } + + /** + * Set the list of users that view permissions on the DAG. If all users are allowed, + * pass in a single entry "*" + * @param users Set of users with view permissions + * @return + */ + public synchronized DAGAccessControls setUsersWithViewACLs(Collection users) { + this.usersWithViewACLs.clear(); + this.usersWithViewACLs.addAll(users); + return this; + } + + /** + * Set the list of users that have modify permissions on the DAG. If all users are allowed, + * pass in a single entry "*" + * @param users Set of users with modify permissions + * @return + */ + public synchronized DAGAccessControls setUsersWithModifyACLs(Collection users) { + this.usersWithModifyACLs.clear(); + this.usersWithModifyACLs.addAll(users); + return this; + } + + /** + * Set the list of groups that have view permissions on the DAG. + * @param groups Set of groups with view permissions + * @return + */ + public synchronized DAGAccessControls setGroupsWithViewACLs(Collection groups) { + this.groupsWithViewACLs.clear(); + this.groupsWithViewACLs.addAll(groups); + return this; + } + + /** + * Set the list of groups that have modify permissions on the DAG. + * @param groups Set of groups with modify permissions + * @return + */ + public synchronized DAGAccessControls setGroupsWithModifyACLs(Collection groups) { + this.groupsWithModifyACLs.clear(); + this.groupsWithModifyACLs.addAll(groups); + return this; + } + + @Private + public Set getUsersWithViewACLs() { + return Collections.unmodifiableSet(usersWithViewACLs); + } + + @Private + public Set getUsersWithModifyACLs() { + return Collections.unmodifiableSet(usersWithModifyACLs); + } + + @Private + public Set getGroupsWithViewACLs() { + return Collections.unmodifiableSet(groupsWithViewACLs); + } + + @Private + public Set getGroupsWithModifyACLs() { + return Collections.unmodifiableSet(groupsWithModifyACLs); + } + + @Private + public void serializeToConfiguration(Configuration conf) { + if (usersWithViewACLs.contains(ACLManager.WILDCARD_ACL_VALUE)) { + conf.set(TezConfiguration.TEZ_DAG_VIEW_ACLS, ACLManager.WILDCARD_ACL_VALUE); + } else { + String userList = ACLManager.toCommaSeparatedString(usersWithViewACLs); + String groupList = ACLManager.toCommaSeparatedString(groupsWithViewACLs); + conf.set(TezConfiguration.TEZ_DAG_VIEW_ACLS, + userList + " " + groupList); + } + if (usersWithModifyACLs.contains(ACLManager.WILDCARD_ACL_VALUE)) { + conf.set(TezConfiguration.TEZ_DAG_MODIFY_ACLS, ACLManager.WILDCARD_ACL_VALUE); + } else { + String userList = ACLManager.toCommaSeparatedString(usersWithModifyACLs); + String groupList = ACLManager.toCommaSeparatedString(groupsWithModifyACLs); + conf.set(TezConfiguration.TEZ_DAG_MODIFY_ACLS, + userList + " " + groupList); + } + } + +} diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java b/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java index 3c54ba76fd..db87d18ac1 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java @@ -41,6 +41,7 @@ import org.apache.hadoop.yarn.api.records.LocalResourceType; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.tez.common.impl.LogUtils; +import org.apache.tez.common.security.DAGAccessControls; import org.apache.tez.dag.api.EdgeProperty.DataMovementType; import org.apache.tez.dag.api.EdgeProperty.DataSourceType; import org.apache.tez.dag.api.EdgeProperty.SchedulingType; @@ -75,6 +76,7 @@ public class DAG { Credentials credentials = new Credentials(); Set vertexGroups = Sets.newHashSet(); Set groupInputEdges = Sets.newHashSet(); + private DAGAccessControls dagAccessControls; public DAG(String name) { this.name = name; @@ -123,6 +125,12 @@ public synchronized Credentials getCredentials() { return this.credentials; } + + public synchronized DAG setAccessControls(DAGAccessControls accessControls) { + this.dagAccessControls = accessControls; + return this; + } + /** * One of the methods that can be used to provide information about required * Credentials when running on a secure cluster. A combination of this and @@ -674,6 +682,10 @@ public DAGPlan createDag(Configuration dagConf) { dagBuilder.addEdge(edgeBuilder); } + if (dagAccessControls != null) { + dagAccessControls.serializeToConfiguration(dagConf); + } + if (dagConf != null) { Iterator> iter = dagConf.iterator(); ConfigurationProto.Builder confProtoBuilder = diff --git a/tez-api/src/test/java/org/apache/tez/common/security/TestACLConfigurationParser.java b/tez-api/src/test/java/org/apache/tez/common/security/TestACLConfigurationParser.java index 6883e12fd4..ae04f76f21 100644 --- a/tez-api/src/test/java/org/apache/tez/common/security/TestACLConfigurationParser.java +++ b/tez-api/src/test/java/org/apache/tez/common/security/TestACLConfigurationParser.java @@ -19,10 +19,7 @@ package org.apache.tez.common.security; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.security.authorize.AccessControlList; -import org.apache.tez.common.security.ACLConfigurationParser; import org.apache.tez.dag.api.TezConfiguration; -import org.apache.tez.common.security.ACLManager.ACLType; import org.junit.Assert; import org.junit.Test; diff --git a/tez-api/src/test/java/org/apache/tez/common/security/TestACLManager.java b/tez-api/src/test/java/org/apache/tez/common/security/TestACLManager.java index aa8c979319..fb8016ba3b 100644 --- a/tez-api/src/test/java/org/apache/tez/common/security/TestACLManager.java +++ b/tez-api/src/test/java/org/apache/tez/common/security/TestACLManager.java @@ -28,10 +28,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.api.records.ApplicationAccessType; -import org.apache.tez.common.security.ACLManager; -import org.apache.tez.common.security.Groups; import org.apache.tez.dag.api.TezConfiguration; -import org.apache.tez.common.security.ACLManager.ACLType; import org.junit.Assert; import org.junit.Test; From 3c4be69d4645602599ca9e2d454daa844d463eb3 Mon Sep 17 00:00:00 2001 From: Hitesh Shah Date: Wed, 13 Aug 2014 14:49:03 -0700 Subject: [PATCH 8/9] DAGAccessControl tests --- .../main/java/org/apache/tez/dag/api/DAG.java | 22 +++-- .../security/TestDAGAccessControls.java | 92 +++++++++++++++++++ .../org/apache/tez/dag/api/TestDAGVerify.java | 45 +++++++++ tez-api/src/test/resources/log4j.properties | 19 ++++ 4 files changed, 172 insertions(+), 6 deletions(-) create mode 100644 tez-api/src/test/java/org/apache/tez/common/security/TestDAGAccessControls.java create mode 100644 tez-api/src/test/resources/log4j.properties diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java b/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java index db87d18ac1..90bfb5193a 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java @@ -682,14 +682,11 @@ public DAGPlan createDag(Configuration dagConf) { dagBuilder.addEdge(edgeBuilder); } - if (dagAccessControls != null) { - dagAccessControls.serializeToConfiguration(dagConf); - } + ConfigurationProto.Builder confProtoBuilder = + ConfigurationProto.newBuilder(); if (dagConf != null) { Iterator> iter = dagConf.iterator(); - ConfigurationProto.Builder confProtoBuilder = - ConfigurationProto.newBuilder(); while (iter.hasNext()) { Entry entry = iter.next(); PlanKeyValuePair.Builder kvp = PlanKeyValuePair.newBuilder(); @@ -697,8 +694,21 @@ public DAGPlan createDag(Configuration dagConf) { kvp.setValue(entry.getValue()); confProtoBuilder.addConfKeyValues(kvp); } - dagBuilder.setDagKeyValues(confProtoBuilder); } + if (dagAccessControls != null) { + Configuration aclConf = new Configuration(false); + dagAccessControls.serializeToConfiguration(aclConf); + Iterator> aclConfIter = aclConf.iterator(); + while (aclConfIter.hasNext()) { + Entry entry = aclConfIter.next(); + PlanKeyValuePair.Builder kvp = PlanKeyValuePair.newBuilder(); + kvp.setKey(entry.getKey()); + kvp.setValue(entry.getValue()); + confProtoBuilder.addConfKeyValues(kvp); + } + } + dagBuilder.setDagKeyValues(confProtoBuilder); + if (credentials != null) { dagBuilder.setCredentialsBinary(DagTypeConverters.convertCredentialsToProto(credentials)); LogUtils.logCredentials(LOG, credentials, "dag"); diff --git a/tez-api/src/test/java/org/apache/tez/common/security/TestDAGAccessControls.java b/tez-api/src/test/java/org/apache/tez/common/security/TestDAGAccessControls.java new file mode 100644 index 0000000000..5c2e46bdc7 --- /dev/null +++ b/tez-api/src/test/java/org/apache/tez/common/security/TestDAGAccessControls.java @@ -0,0 +1,92 @@ +/** + * 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.tez.common.security; + +import java.util.Arrays; + +import org.apache.hadoop.conf.Configuration; +import org.apache.tez.dag.api.TezConfiguration; +import org.junit.Assert; +import org.junit.Test; + +public class TestDAGAccessControls { + + @Test + public void testBasicSerializeToConf() { + DAGAccessControls dagAccessControls = new DAGAccessControls(); + dagAccessControls.setUsersWithViewACLs(Arrays.asList("u1")) + .setUsersWithModifyACLs(Arrays.asList("u2")) + .setGroupsWithViewACLs(Arrays.asList("g1")) + .setGroupsWithModifyACLs(Arrays.asList("g2")); + + Configuration conf = new Configuration(false); + dagAccessControls.serializeToConfiguration(conf); + Assert.assertNotNull(conf.get(TezConfiguration.TEZ_DAG_VIEW_ACLS)); + Assert.assertNotNull(conf.get(TezConfiguration.TEZ_DAG_MODIFY_ACLS)); + + Assert.assertEquals("u1 g1", conf.get(TezConfiguration.TEZ_DAG_VIEW_ACLS)); + Assert.assertEquals("u2 g2", conf.get(TezConfiguration.TEZ_DAG_MODIFY_ACLS)); + } + + @Test + public void testWildCardSerializeToConf() { + DAGAccessControls dagAccessControls = new DAGAccessControls(); + dagAccessControls.setUsersWithViewACLs(Arrays.asList("*")) + .setUsersWithModifyACLs(Arrays.asList("*")) + .setGroupsWithViewACLs(Arrays.asList("g1")) + .setGroupsWithModifyACLs(Arrays.asList("g2")); + + Configuration conf = new Configuration(false); + dagAccessControls.serializeToConfiguration(conf); + Assert.assertNotNull(conf.get(TezConfiguration.TEZ_DAG_VIEW_ACLS)); + Assert.assertNotNull(conf.get(TezConfiguration.TEZ_DAG_MODIFY_ACLS)); + + Assert.assertEquals("*", conf.get(TezConfiguration.TEZ_DAG_VIEW_ACLS)); + Assert.assertEquals("*", conf.get(TezConfiguration.TEZ_DAG_MODIFY_ACLS)); + } + + @Test + public void testGroupsOnlySerializeToConf() { + DAGAccessControls dagAccessControls = new DAGAccessControls(); + dagAccessControls.setGroupsWithViewACLs(Arrays.asList("g1")) + .setGroupsWithModifyACLs(Arrays.asList("g2")); + + Configuration conf = new Configuration(false); + dagAccessControls.serializeToConfiguration(conf); + Assert.assertNotNull(conf.get(TezConfiguration.TEZ_DAG_VIEW_ACLS)); + Assert.assertNotNull(conf.get(TezConfiguration.TEZ_DAG_MODIFY_ACLS)); + + Assert.assertEquals(" g1", conf.get(TezConfiguration.TEZ_DAG_VIEW_ACLS)); + Assert.assertEquals(" g2", conf.get(TezConfiguration.TEZ_DAG_MODIFY_ACLS)); + } + + @Test + public void testEmptySerializeToConf() { + DAGAccessControls dagAccessControls = new DAGAccessControls(); + + Configuration conf = new Configuration(false); + dagAccessControls.serializeToConfiguration(conf); + Assert.assertNotNull(conf.get(TezConfiguration.TEZ_DAG_VIEW_ACLS)); + Assert.assertNotNull(conf.get(TezConfiguration.TEZ_DAG_MODIFY_ACLS)); + + Assert.assertEquals(" ", conf.get(TezConfiguration.TEZ_DAG_VIEW_ACLS)); + Assert.assertEquals(" ", conf.get(TezConfiguration.TEZ_DAG_MODIFY_ACLS)); + } + +} diff --git a/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGVerify.java b/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGVerify.java index d892ca177e..61ac3665dc 100644 --- a/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGVerify.java +++ b/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGVerify.java @@ -18,10 +18,17 @@ package org.apache.tez.dag.api; +import java.util.Arrays; + +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.tez.common.security.DAGAccessControls; import org.apache.tez.dag.api.EdgeProperty.DataMovementType; import org.apache.tez.dag.api.EdgeProperty.DataSourceType; import org.apache.tez.dag.api.EdgeProperty.SchedulingType; +import org.apache.tez.dag.api.records.DAGProtos.ConfigurationProto; +import org.apache.tez.dag.api.records.DAGProtos.DAGPlan; +import org.apache.tez.dag.api.records.DAGProtos.PlanKeyValuePair; import org.junit.Assert; import org.junit.Test; @@ -888,4 +895,42 @@ public void testMultipleRootInputsAllowed() { dag.createDag(new TezConfiguration()); } + + @Test(timeout = 5000) + public void testDAGAccessControls() { + DAG dag = new DAG("testDag"); + ProcessorDescriptor pd1 = new ProcessorDescriptor("processor1") + .setUserPayload(new UserPayload("processor1Bytes".getBytes())); + Vertex v1 = new Vertex("v1", pd1, 10, Resource.newInstance(1024, 1)); + dag.addVertex(v1); + + DAGAccessControls dagAccessControls = new DAGAccessControls(); + dagAccessControls.setUsersWithViewACLs(Arrays.asList("u1")) + .setUsersWithModifyACLs(Arrays.asList("*")) + .setGroupsWithViewACLs(Arrays.asList("g1")) + .setGroupsWithModifyACLs(Arrays.asList("g2")); + dag.setAccessControls(dagAccessControls); + + Configuration conf = new Configuration(false); + DAGPlan dagPlan = dag.createDag(conf); + Assert.assertNull(conf.get(TezConfiguration.TEZ_DAG_VIEW_ACLS)); + Assert.assertNull(conf.get(TezConfiguration.TEZ_DAG_MODIFY_ACLS)); + + ConfigurationProto confProto = dagPlan.getDagKeyValues(); + boolean foundViewAcls = false; + boolean foundModifyAcls = false; + + for (PlanKeyValuePair pair : confProto.getConfKeyValuesList()) { + if (pair.getKey().equals(TezConfiguration.TEZ_DAG_VIEW_ACLS)) { + foundViewAcls = true; + Assert.assertEquals("u1 g1", pair.getValue()); + } else if (pair.getKey().equals(TezConfiguration.TEZ_DAG_MODIFY_ACLS)) { + foundModifyAcls = true; + Assert.assertEquals("*", pair.getValue()); + } + } + Assert.assertTrue(foundViewAcls); + Assert.assertTrue(foundModifyAcls); + } + } diff --git a/tez-api/src/test/resources/log4j.properties b/tez-api/src/test/resources/log4j.properties new file mode 100644 index 0000000000..531b68b5a9 --- /dev/null +++ b/tez-api/src/test/resources/log4j.properties @@ -0,0 +1,19 @@ +# Licensed 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. + +# log4j configuration used during build and unit tests + +log4j.rootLogger=info,stdout +log4j.threshhold=ALL +log4j.appender.stdout=org.apache.log4j.ConsoleAppender +log4j.appender.stdout.layout=org.apache.log4j.PatternLayout +log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %c{2} (%F:%M(%L)) - %m%n From d93b12b1c2528b75ec2ff444d4bc7ffcab0ca3c8 Mon Sep 17 00:00:00 2001 From: Hitesh Shah Date: Wed, 13 Aug 2014 15:18:06 -0700 Subject: [PATCH 9/9] Javadocs --- tez-api/src/main/java/org/apache/tez/dag/api/DAG.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java b/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java index 90bfb5193a..47792592a3 100644 --- a/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java +++ b/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java @@ -126,6 +126,14 @@ public synchronized Credentials getCredentials() { } + /** + * Set Access controls for the DAG. Which user/groups can view the DAG progess/history and + * who can modify the DAG i.e. kill the DAG. + * The owner of the Tez Session and the user submitting the DAG are super-users and have access + * to all operations on the DAG. + * @param accessControls Access Controls + * @return + */ public synchronized DAG setAccessControls(DAGAccessControls accessControls) { this.dagAccessControls = accessControls; return this;