소스 검색

YARN-8838. Check that container user is same as websocket user for interactive shell. Contributed by Eric Yang

Billie Rinaldi 6 년 전
부모
커밋
49824ed260

+ 4 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/YarnClientImpl.java

@@ -1105,6 +1105,10 @@ public class YarnClientImpl extends YarnClient {
       WebSocketClient client = new WebSocketClient();
       URI uri = URI.create(protocol + host + ":" + port + "/container/" +
           containerId);
+      if (!UserGroupInformation.isSecurityEnabled()) {
+        uri = URI.create(protocol + host + ":" + port + "/container/" +
+            containerId + "?user.name=" + System.getProperty("user.name"));
+      }
       try {
         client.start();
         // The socket that receives events

+ 41 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerShellWebSocket.java

@@ -21,6 +21,8 @@ package org.apache.hadoop.yarn.server.nodemanager.webapp;
 import java.io.IOException;
 import java.net.URI;
 import java.nio.charset.Charset;
+import java.util.List;
+import java.util.Map;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -35,6 +37,8 @@ import org.eclipse.jetty.websocket.api.annotations.OnWebSocketConnect;
 import org.eclipse.jetty.websocket.api.annotations.OnWebSocketMessage;
 import org.eclipse.jetty.websocket.api.annotations.WebSocket;
 import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
+import org.apache.hadoop.security.HadoopKerberosName;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -92,14 +96,16 @@ public class ContainerShellWebSocket {
 
   @OnWebSocketConnect
   public void onConnect(Session session) {
-    LOG.info(session.getRemoteAddress().getHostString() + " connected!");
-
     try {
       URI containerURI = session.getUpgradeRequest().getRequestURI();
       String[] containerPath = containerURI.getPath().split("/");
       String cId = containerPath[2];
       Container container = nmContext.getContainers().get(ContainerId
           .fromString(cId));
+      if (!checkAuthorization(session, container)) {
+        session.close(1008, "Forbidden");
+      }
+      LOG.info(session.getRemoteAddress().getHostString() + " connected!");
       LOG.info(
           "Making interactive connection to running docker container with ID: "
               + cId);
@@ -126,4 +132,37 @@ public class ContainerShellWebSocket {
     }
   }
 
+  /**
+   * Check if user is authorized to access container.
+   * @param session websocket session
+   * @param container instance of container to access
+   * @return true if user is allowed to access container.
+   * @throws IOException
+   */
+  protected boolean checkAuthorization(Session session, Container container)
+      throws IOException {
+    boolean authorized = true;
+    String user = "";
+    if (UserGroupInformation.isSecurityEnabled()) {
+      user = new HadoopKerberosName(session.getUpgradeRequest()
+          .getUserPrincipal().getName()).getShortName();
+    } else {
+      Map<String, List<String>> parameters = session.getUpgradeRequest()
+          .getParameterMap();
+      if (parameters.containsKey("user.name")) {
+        List<String> users = parameters.get("user.name");
+        user = users.get(0);
+      }
+    }
+    boolean isAdmin = false;
+    if (nmContext.getApplicationACLsManager().areACLsEnabled()) {
+      UserGroupInformation ugi = UserGroupInformation.createRemoteUser(user);
+      isAdmin = nmContext.getApplicationACLsManager().isAdmin(ugi);
+    }
+    String containerUser = container.getUser();
+    if (!user.equals(containerUser) && !isAdmin) {
+      authorized = false;
+    }
+    return authorized;
+  }
 }

+ 38 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMContainerWebSocket.java

@@ -27,17 +27,25 @@ import org.apache.hadoop.yarn.server.nodemanager.LocalDirsHandlerService;
 import org.apache.hadoop.yarn.server.nodemanager.NodeHealthCheckerService;
 import org.apache.hadoop.yarn.server.nodemanager.NodeManager;
 import org.apache.hadoop.yarn.server.nodemanager.ResourceView;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.eclipse.jetty.websocket.api.Session;
+import org.eclipse.jetty.websocket.api.UpgradeRequest;
 import org.eclipse.jetty.websocket.client.WebSocketClient;
 import org.junit.After;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
+import static org.mockito.Mockito.*;
 import java.io.File;
+import java.io.IOException;
 import java.net.URI;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 import java.util.concurrent.Future;
 
 /**
@@ -51,6 +59,7 @@ public class TestNMContainerWebSocket {
       TestNMWebServer.class.getSimpleName());
   private static File testLogDir = new File("target",
       TestNMWebServer.class.getSimpleName() + "LogDir");
+  private WebServer server;
 
   @Before
   public void setup() {
@@ -101,7 +110,7 @@ public class TestNMContainerWebSocket {
     healthChecker.init(conf);
     LocalDirsHandlerService dirsHandler = healthChecker.getDiskHandler();
     conf.set(YarnConfiguration.NM_WEBAPP_ADDRESS, webAddr);
-    WebServer server = new WebServer(nmContext, resourceView,
+    server = new WebServer(nmContext, resourceView,
         new ApplicationACLsManager(conf), dirsHandler);
     try {
       server.init(conf);
@@ -141,9 +150,36 @@ public class TestNMContainerWebSocket {
     } finally {
       try {
         client.stop();
+        server.close();
       } catch (Exception e) {
         LOG.error("Failed to close client", e);
       }
     }
   }
+
+  @Test
+  public void testContainerShellWebSocket() {
+    Context nm = mock(Context.class);
+    Session session = mock(Session.class);
+    Container container = mock(Container.class);
+    UpgradeRequest request = mock(UpgradeRequest.class);
+    ApplicationACLsManager aclManager = mock(ApplicationACLsManager.class);
+    ContainerShellWebSocket.init(nm);
+    ContainerShellWebSocket ws = new ContainerShellWebSocket();
+    List<String> names = new ArrayList<>();
+    names.add("foobar");
+    Map<String, List<String>> mockParameters = new HashMap<>();
+    mockParameters.put("user.name", names);
+    when(session.getUpgradeRequest()).thenReturn(request);
+    when(request.getParameterMap()).thenReturn(mockParameters);
+    when(container.getUser()).thenReturn("foobar");
+    when(nm.getApplicationACLsManager()).thenReturn(aclManager);
+    when(aclManager.areACLsEnabled()).thenReturn(false);
+    try {
+      boolean authorized = ws.checkAuthorization(session, container);
+      Assert.assertTrue("Not authorized", authorized);
+    } catch (IOException e) {
+      Assert.fail("Should not throw exception.");
+    }
+  }
 }