浏览代码

HUE-219. Secure plugins with SASL

* Plugins now use HadoopThriftAuthBridge to connect UGI based credentials to Thrift's SASL
* Adds Thrift SASL transport on the Python side
* Also updates plugins to use the hue user as the proxy user now that hue is authenticated via SASL
* Hue hadoopfs implementation takes kerberos principal of NN as a parameter, connects with it
Todd Lipcon 15 年之前
父节点
当前提交
0297cae05d

+ 130 - 0
desktop/core/src/desktop/lib/thrift_sasl.py

@@ -0,0 +1,130 @@
+#!/usr/bin/env python
+
+#
+# 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 sys
+
+from cStringIO import StringIO
+from thrift.transport import TTransport
+from thrift.transport.TTransport import *
+from thrift.protocol import TBinaryProtocol
+import sasl
+import struct
+
+class TSaslClientTransport(TTransportBase):
+  START = 1
+  OK = 2
+  BAD = 3
+  ERROR = 4
+  COMPLETE = 5
+
+  def __init__(self, sasl_client_factory, mechanism, trans):
+    self._trans = trans
+    self.sasl_client_factory = sasl_client_factory
+    self.sasl = None
+    self.mechanism = mechanism
+    self.__wbuf = StringIO()
+    self.__rbuf = StringIO()
+    self.opened = False
+
+  def isOpen(self):
+    return self._trans.isOpen()
+
+  def open(self):
+    if not self._trans.isOpen():
+      self._trans.open()
+
+    if self.sasl is not None:
+      raise TTransportException(
+        type=TTransportException.NOT_OPEN,
+        message="Already open!")
+    self.sasl = self.sasl_client_factory()
+
+    ret, chosen_mech, initial_response = self.sasl.start(self.mechanism)
+    if not ret:
+      raise TTransportException(type=TTransportException.NOT_OPEN,
+        message=("Could not start SASL: %s" % self.sasl.getError()))
+
+    # Send initial response
+    self._send_message(self.START, chosen_mech)
+    self._send_message(self.OK, initial_response)
+
+    # SASL negotiation loop
+    while True:
+      status, payload = self._recv_sasl_message()
+      if status not in (self.OK, self.COMPLETE):
+        raise TTransportException(type=TTransportException.NOT_OPEN,
+          message=("Bad status: %d (%s)" % (status, payload)))
+      if status == self.COMPLETE:
+        break
+      ret, response = self.sasl.step(payload)
+      if not ret:
+        raise TTransportException(type=TTransportException.NOT_OPEN,
+          message=("Bad SASL result: %s" % (self.sasl.getError())))
+      self._send_message(self.OK, response)
+
+  def _send_message(self, status, body):
+    header = struct.pack(">BI", status, len(body))
+    self._trans.write(header + body)
+    self._trans.flush()
+
+  def _recv_sasl_message(self):
+    header = self._trans.read(5)
+    status, length = struct.unpack(">BI", header)
+    if length > 0:
+      payload = self._trans.read(length)
+    else:
+      payload = ""
+    return status, payload
+
+  def write(self, data):
+    self.__wbuf.write(data)
+
+  def flush(self):
+    success, encoded = self.sasl.encode(self.__wbuf.getvalue())
+    if not success:
+      raise TTransportException(type=TTransportException.UNKNOWN,
+                                message=self.sasl.getError())
+    # Note stolen from TFramedTransport:
+    # N.B.: Doing this string concatenation is WAY cheaper than making
+    # two separate calls to the underlying socket object. Socket writes in
+    # Python turn out to be REALLY expensive, but it seems to do a pretty
+    # good job of managing string buffer operations without excessive copies
+    self._trans.write(struct.pack(">I", len(encoded)) + encoded)
+    self._trans.flush()
+    self.__wbuf = StringIO()
+
+  def read(self, sz):
+    ret = self.__rbuf.read(sz)
+    if len(ret) != 0:
+      return ret
+
+    self._read_frame()
+    return self.__rbuf.read(sz)
+
+  def _read_frame(self):
+    header = self._trans.readAll(4)
+    (length,) = struct.unpack(">I", header)
+    self.__rbuf = StringIO(self._trans.readAll(length))
+
+
+  def close(self):
+    self._trans.close()
+    self.sasl = None

+ 40 - 11
desktop/core/src/desktop/lib/thrift_util.py

@@ -20,6 +20,7 @@
 import socket
 import logging
 import Queue
+import sasl
 import select
 import sys
 import threading
@@ -30,6 +31,7 @@ from thrift.transport.TSocket import TSocket
 from thrift.transport.TTransport import TBufferedTransport, TMemoryBuffer,\
                                         TTransportException
 from thrift.protocol.TBinaryProtocol import TBinaryProtocol
+from desktop.lib.thrift_sasl import TSaslClientTransport
 
 # The maximum depth that we will recurse through a "jsonable" structure
 # while converting to thrift. This prevents us from infinite recursion
@@ -64,6 +66,7 @@ class ConnectionPooler(object):
     self.dictlock = threading.Lock()
 
   def get_client(self, klass, host, port, service_name="Unknown",
+                 kerberos_principal="thrift",
                  get_client_timeout=None):
     """
     Could block while we wait for the pool to become non-empty.
@@ -71,7 +74,6 @@ class ConnectionPooler(object):
     @param get_client_timeout: how long (in seconds) to wait on the pool
                                to get a client before failing
     """
-
     # First up, check to see if we have a pool for this endpoint
     if (host,port) not in self.pooldict:
       # Uh-oh, we need to initialise the queue. Take the dict lock.
@@ -93,7 +95,9 @@ class ConnectionPooler(object):
           q = Queue.Queue(self.poolsize)
           self.pooldict[(host, port)] = q
           for i in xrange(self.poolsize):
-            client = construct_client(klass, host, port, service_name)
+            client = construct_client(klass, host, port,
+                                      service_name=service_name,
+                                      kerberos_principal=kerberos_principal)
             client.CID = i
             q.put(client, False)
       finally:
@@ -131,33 +135,56 @@ class ConnectionPooler(object):
     """
     self.pooldict[(host, port)].put(client)
 
-def construct_client(klass, host, port, service_name, timeout_seconds=45):
+def construct_client(klass, host, port, service_name, kerberos_principal="thrift", timeout_seconds=45):
   """
   Constructs a thrift client, lazily.
   """
+
+  def sasl_factory():
+    saslc = sasl.Client()
+    saslc.setAttr("host", host)
+    saslc.setAttr("service", kerberos_principal)
+    saslc.init()
+    return saslc
+
+  logging.info("service: %s   host: %s" % (kerberos_principal, host))
   sock = TSocket(host, port)
   if timeout_seconds:
     # Thrift trivia: You can do this after the fact with
     # self.wrapped.transport._TBufferedTransport__trans.setTimeout(seconds*1000)
     sock.setTimeout(timeout_seconds*1000.0)
-  transport = TBufferedTransport(sock)
+  transport = TSaslClientTransport(sasl_factory, "GSSAPI", sock)
   protocol = TBinaryProtocol(transport)
   service = klass(protocol)
   return SuperClient(service, transport, timeout_seconds=timeout_seconds)
 
 _connection_pool = ConnectionPooler()
 
-def get_client(klass, host, port, service_name, timeout_seconds=None):
-  return PooledClient(klass,host,port,service_name,timeout_seconds)
+def get_client(klass, host, port, service_name, kerberos_principal="thrift", timeout_seconds=None):
+  return PooledClient(klass,host,port,service_name,
+                      kerberos_principal=kerberos_principal,
+                      timeout_seconds=timeout_seconds)
+
+def _grab_transport_from_wrapper(outer_transport):
+  if isinstance(outer_transport, TBufferedTransport):
+    return outer_transport._TBufferedTransport__trans
+  elif isinstance(outer_transport, TSaslClientTransport):
+    return outer_transport._trans
+  else:
+    raise Exception("Unknown transport type: " + outer_transport.__class__)
 
 class PooledClient(object):
   """
   A wrapper for a SuperClient
   """
-  def __init__(self, klass, host, port, service_name = "Unknown", timeout_seconds=None):
+  def __init__(self, klass, host, port,
+               service_name = "Unknown",
+               kerberos_principal="thrift",
+               timeout_seconds=None):
     self.klass = klass
     self.host = host
     self.port = port
+    self.kerberos_principal = kerberos_principal
     self.timeout_seconds = timeout_seconds
     self.service_name = service_name
 
@@ -167,7 +194,9 @@ class PooledClient(object):
 
     # Fetch the thrift client from the pool
     superclient = _connection_pool.get_client(self.klass, self.host, self.port,
-                                              get_client_timeout=self.timeout_seconds)
+                                              kerberos_principal=self.kerberos_principal,
+                                              get_client_timeout=self.timeout_seconds,
+                                              service_name=self.service_name)
 
     res = getattr(superclient, attr)
     if hasattr(res,"__call__"):
@@ -176,7 +205,7 @@ class PooledClient(object):
           try:
             # Poke it to see if it's closed on the other end. This can happen if a connection
             # sits in the connection pool longer than the read timeout of the server.
-            sock = superclient.transport._TBufferedTransport__trans.handle
+            sock = _grab_transport_from_wrapper(superclient.transport).handle
             if sock:
               rlist,wlist,xlist = select.select([sock], [], [], 0)
               if rlist:
@@ -277,9 +306,9 @@ class SuperClient(object):
       self.timeout_seconds = timeout_seconds
       # ugh, None is a valid timeout
       if self.timeout_seconds is not None:
-        self.transport._TBufferedTransport__trans.setTimeout(self.timeout_seconds * 1000)
+        _grab_transport_from_wrapper(self.transport).setTimeout(self.timeout_seconds * 1000)
       else:
-        self.transport._TBufferedTransport__trans.setTimeout(None)
+        _grab_transport_from_wrapper(self.transport).setTimeout(None)
 
 def simpler_string(thrift_obj):
   """

+ 3 - 3
desktop/libs/hadoop/java/src/java/org/apache/hadoop/mapred/ThriftJobTrackerPlugin.java

@@ -658,9 +658,9 @@ public class ThriftJobTrackerPlugin extends JobTrackerPlugin implements Configur
           // The port may have been 0, so we update it.
           conf.set(THRIFT_ADDRESS_PROPERTY, address.getHostName() + ":" +
               thriftServer.getPort());
-        } catch (java.io.IOException ioe) {
-            LOG.warn("Cannot start Thrift jobtracker plug-in",ioe);
-            throw new RuntimeException("Cannot start Thrift jobtracker plug-in", ioe);
+        } catch (Exception  e) {
+            LOG.warn("Cannot start Thrift jobtracker plug-in", e);
+            throw new RuntimeException("Cannot start Thrift jobtracker plug-in", e);
         }
     }
 

+ 4 - 4
desktop/libs/hadoop/java/src/java/org/apache/hadoop/thriftfs/DatanodePlugin.java

@@ -166,12 +166,12 @@ public class DatanodePlugin
       InetSocketAddress address = NetUtils.createSocketAddr(
         conf.get(THRIFT_ADDRESS_PROPERTY, DEFAULT_THRIFT_ADDRESS));
 
-      thriftServer = new ThriftPluginServer(address,
-                                            new ProcessorFactory());
+      thriftServer = new ThriftPluginServer(
+        address, new ProcessorFactory());
       thriftServer.setConf(conf);
       thriftServer.start();
-    } catch (java.io.IOException ioe) {
-      throw new RuntimeException("Could not start Thrift Datanode Plugin", ioe);
+    } catch (Exception e) {
+      throw new RuntimeException("Could not start Thrift Datanode Plugin", e);
     }
   }
 

+ 385 - 0
desktop/libs/hadoop/java/src/java/org/apache/hadoop/thriftfs/HadoopThriftAuthBridge.java

@@ -0,0 +1,385 @@
+/**
+ * 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.hadoop.thriftfs;
+
+import java.security.PrivilegedAction;
+import java.security.PrivilegedExceptionAction;
+import javax.security.sasl.SaslException;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.security.SaslRpcServer.AuthMethod;
+import org.apache.hadoop.security.*;
+import org.apache.thrift.TException;
+import org.apache.thrift.transport.TSaslClientTransport;
+import org.apache.thrift.transport.TSaslServerTransport;
+import org.apache.thrift.transport.TTransport;
+import org.apache.thrift.TProcessor;
+import org.apache.thrift.TProcessorFactory;
+import org.apache.thrift.protocol.TProtocol;
+import org.apache.thrift.transport.TTransportException;
+import org.apache.thrift.transport.TTransportFactory;
+
+
+/**
+ * Functions that bridge Thrift's SASL transports to Hadoop's
+ * SASL callback handlers and authentication classes.
+ *
+ * This is borrowed from Hive's metastore.
+ */
+class HadoopThriftAuthBridge {
+  static final Log LOG = LogFactory.getLog(HadoopThriftAuthBridge.class);
+
+  public static class Client {
+    /**
+     * Create a client-side SASL transport that wraps an underlying transport.
+     *
+     * @param method The authentication method to use. Currently only KERBEROS is
+     *               supported.
+     * @param serverPrincipal The Kerberos principal of the target server.
+     * @param underlyingTransport The underlying transport mechanism, usually a TSocket.
+     */
+    public TTransport createClientTransport(
+      String principalConfig, String host,
+      String methodStr, TTransport underlyingTransport)
+      throws IOException {
+      AuthMethod method = AuthMethod.valueOf(AuthMethod.class, methodStr);
+
+      switch (method) {
+        case KERBEROS:
+          String serverPrincipal = SecurityUtil.getServerPrincipal(principalConfig, host);
+          String names[] = SaslRpcServer.splitKerberosName(serverPrincipal);
+          if (names.length != 3) {
+            throw new IOException(
+              "Kerberos principal name does NOT have the expected hostname part: "
+              + serverPrincipal);
+          }
+          try {
+            TTransport saslTransport = new TSaslClientTransport(
+              method.getMechanismName(),
+              null,
+              names[0], names[1],
+              SaslRpcServer.SASL_PROPS, null,
+              underlyingTransport);
+            return new TUGIAssumingTransport(saslTransport, UserGroupInformation.getCurrentUser());
+          } catch (SaslException se) {
+            throw new IOException("Could not instantiate SASL transport", se);
+          }
+
+        default:
+          throw new IOException("Unsupported authentication method: " + method);
+      }
+    }
+  }
+
+  public static class Server {
+    private final UserGroupInformation realUgi;
+
+    /**
+     * TODO: javadoc
+     */
+    public Server(String keytabFile, String principalConf)
+      throws TTransportException {
+      if (keytabFile == null || keytabFile.isEmpty()) {
+        throw new TTransportException("No keytab specified");
+      }
+      if (principalConf == null || principalConf.isEmpty()) {
+        throw new TTransportException("No principal specified");
+      }
+
+      // Login from the keytab
+      UserGroupInformation ugi;
+      String kerberosName;
+      try {
+        kerberosName = SecurityUtil.getServerPrincipal(
+          principalConf, null);
+        realUgi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(
+          kerberosName, keytabFile);
+        assert realUgi.isFromKeytab();
+      } catch (IOException ioe) {
+        throw new TTransportException(ioe);
+      }
+    }
+
+    public Server() throws TTransportException {
+      try {
+        realUgi = UserGroupInformation.getLoginUser();
+      } catch (IOException ioe) {
+        throw new TTransportException(ioe);
+      }
+      if (realUgi == null || !realUgi.hasKerberosCredentials()) {
+        throw new TTransportException("UGI " + realUgi + " has no kerberos credentials");
+      }
+
+      if (!realUgi.isFromKeytab()) {
+        LOG.warn("Thrift server starting with a non-keytab login user: " + realUgi);
+      }
+    }
+
+    /**
+     * Create a TTransportFactory that, upon connection of a client socket,
+     * negotiates a Kerberized SASL transport. The resulting TTransportFactory
+     * can be passed as both the input and output transport factory when
+     * instantiating a TThreadPoolServer, for example.
+     *
+     */
+    public TTransportFactory createTransportFactory() throws TTransportException
+    {
+      // Parse out the kerberos principal, host, realm.
+      String kerberosName = realUgi.getUserName();
+      final String names[] = SaslRpcServer.splitKerberosName(kerberosName);
+      if (names.length != 3) {
+        throw new TTransportException("Kerberos principal should have 3 parts: " + kerberosName);
+      }
+
+      TSaslServerTransport.Factory transFactory = new TSaslServerTransport.Factory();
+      transFactory.addServerDefinition(
+        AuthMethod.KERBEROS.getMechanismName(),
+        names[0], names[1],  // two parts of kerberos principal
+        SaslRpcServer.SASL_PROPS,
+        new SaslRpcServer.SaslGssCallbackHandler());
+
+      return new TUGIAssumingTransportFactory(transFactory, realUgi);
+    }
+
+    /**
+     * Wrap a TProcessor in such a way that, before processing any RPC, it
+     * assumes the UserGroupInformation of the user authenticated by
+     * the SASL transport.
+     */
+    public TProcessor wrapProcessor(TProcessor processor) {
+      return new TUGIAssumingProcessor(processor);
+    }
+
+    public TProcessorFactory wrapProcessorFactory(final TProcessorFactory factory) {
+      return new TProcessorFactory(null) {
+        @Override
+        public TProcessor getProcessor(TTransport trans) {
+          return wrapProcessor(factory.getProcessor(trans));
+        }
+      };
+    }
+
+    /**
+     * Processor that pulls the SaslServer object out of the transport, and
+     * assumes the remote user's UGI before calling through to the original
+     * processor.
+     *
+     * This is used on the server side to set the UGI for each specific call.
+     */
+    private class TUGIAssumingProcessor implements TProcessor {
+      final TProcessor wrapped;
+
+      TUGIAssumingProcessor(TProcessor wrapped) {
+        this.wrapped = wrapped;
+      }
+
+      public boolean process(final TProtocol inProt, final TProtocol outProt) throws TException {
+        TTransport trans = inProt.getTransport();
+        if (!(trans instanceof TSaslServerTransport)) {
+          throw new TException("Unexpected non-SASL transport " + trans.getClass());
+        }
+        TSaslServerTransport saslTrans = (TSaslServerTransport)trans;
+        String authId = saslTrans.getSaslServer().getAuthorizationID();
+        System.err.println("AUTH ID ======>" + authId);
+
+        UserGroupInformation clientUgi = UserGroupInformation.createRemoteUser(authId);
+
+        try {
+          return clientUgi.doAs(new PrivilegedExceptionAction<Boolean>() {
+              public Boolean run() {
+                try {
+                  return wrapped.process(inProt, outProt);
+                } catch (TException te) {
+                  throw new RuntimeException(te);
+                }
+              }
+            });
+        } catch (RuntimeException rte) {
+          if (rte.getCause() instanceof TException) {
+            throw (TException)rte.getCause();
+          }
+          throw rte;
+        } catch (InterruptedException ie) {
+          throw new RuntimeException(ie); // unexpected!
+        } catch (IOException ioe) {
+          throw new RuntimeException(ioe); // unexpected!
+        }
+      }
+    }
+
+  }
+
+  /**
+   * A TransportFactory that wraps another one, but assumes a specified UGI
+   * before calling through.
+   *
+   * This is used on the server side to assume the server's Principal when accepting
+   * clients.
+   */
+  private static class TUGIAssumingTransportFactory extends TTransportFactory {
+    private final UserGroupInformation ugi;
+    private final TTransportFactory wrapped;
+
+    public TUGIAssumingTransportFactory(TTransportFactory wrapped, UserGroupInformation ugi) {
+      assert wrapped != null;
+      assert ugi != null;
+
+      this.wrapped = wrapped;
+      this.ugi = ugi;
+    }
+
+    @Override
+    public TTransport getTransport(final TTransport trans) {
+      return ugi.doAs(new PrivilegedAction<TTransport>() {
+        public TTransport run() {
+          return wrapped.getTransport(trans);
+        }
+      });
+    }
+  }
+
+  /**
+   * The Thrift SASL transports call Sasl.createSaslServer and Sasl.createSaslClient
+   * inside open(). So, we need to assume the correct UGI when the transport is opened
+   * so that the SASL mechanisms have access to the right principal. This transport
+   * wraps the Sasl transports to set up the right UGI context for open().
+   *
+   * This is used on the client side, where the API explicitly opens a transport to
+   * the server.
+   */
+  private static class TUGIAssumingTransport extends TFilterTransport {
+    private final UserGroupInformation ugi;
+
+    public TUGIAssumingTransport(TTransport wrapped, UserGroupInformation ugi) {
+      super(wrapped);
+      this.ugi = ugi;
+    }
+
+    @Override
+    public void open() throws TTransportException {
+      try {
+        ugi.doAs(new PrivilegedExceptionAction<Void>() {
+          public Void run() {
+            try {
+              wrapped.open();
+            } catch (TTransportException tte) {
+              // Wrap the transport exception in an RTE, since UGI.doAs() then goes
+              // and unwraps this for us out of the doAs block. We then unwrap one
+              // more time in our catch clause to get back the TTE. (ugh)
+              throw new RuntimeException(tte);
+            }
+            return null;
+          }
+        });
+      } catch (IOException ioe) {
+        assert false : "Never thrown!";
+        throw new RuntimeException("Received an ioe we never threw!", ioe);
+      } catch (InterruptedException ie) {
+        assert false : "We never expect to see an InterruptedException thrown in this block";
+        throw new RuntimeException("Received an ie we never threw!", ie);
+      } catch (RuntimeException rte) {
+        if (rte.getCause() instanceof TTransportException) {
+          throw (TTransportException)rte.getCause();
+        } else {
+          throw rte;
+        }
+      }
+    }
+  }
+
+  /**
+   * Transport that simply wraps another transport.
+   * This is the equivalent of FilterInputStream for Thrift transports.
+   */
+  private static class TFilterTransport extends TTransport {
+    protected final TTransport wrapped;
+
+    public TFilterTransport(TTransport wrapped) {
+      this.wrapped = wrapped;
+    }
+
+    @Override
+    public void open() throws TTransportException {
+      wrapped.open();
+    }
+
+    @Override
+    public boolean isOpen() {
+      return wrapped.isOpen();
+    }
+
+    @Override
+    public boolean peek() {
+      return wrapped.peek();
+    }
+
+    @Override
+    public void close() {
+      wrapped.close();
+    }
+
+    @Override
+    public int read(byte[] buf, int off, int len) throws TTransportException {
+      return wrapped.read(buf, off, len);
+    }
+
+    @Override
+    public int readAll(byte[] buf, int off, int len) throws TTransportException {
+      return wrapped.readAll(buf, off, len);
+    }
+
+    @Override
+    public void write(byte[] buf) throws TTransportException {
+      wrapped.write(buf);
+    }
+
+    @Override
+    public void write(byte[] buf, int off, int len) throws TTransportException {
+      wrapped.write(buf, off, len);
+    }
+
+    @Override
+    public void flush() throws TTransportException {
+      wrapped.flush();
+    }
+
+    @Override
+    public byte[] getBuffer() {
+      return wrapped.getBuffer();
+    }
+
+    @Override
+    public int getBufferPosition() {
+      return wrapped.getBufferPosition();
+    }
+
+    @Override
+    public int getBytesRemainingInBuffer() {
+      return wrapped.getBytesRemainingInBuffer();
+    }
+
+    @Override
+    public void consumeBuffer(int len) {
+      wrapped.consumeBuffer(len);
+    }
+  }
+}

+ 2 - 2
desktop/libs/hadoop/java/src/java/org/apache/hadoop/thriftfs/NamenodePlugin.java

@@ -418,8 +418,8 @@ public class NamenodePlugin extends org.apache.hadoop.hdfs.server.namenode.Namen
       thriftServer.start();
       // The port may have been 0, so we update it.
       conf.set(THRIFT_ADDRESS_PROPERTY, address.getHostName() + ":" + thriftServer.getPort());
-    } catch (java.io.IOException ioe) {
-      throw new RuntimeException("Cannot start Thrift namenode plug-in", ioe);
+    } catch (Exception e) {
+      throw new RuntimeException("Cannot start Thrift namenode plug-in", e);
     }
   }
 

+ 4 - 2
desktop/libs/hadoop/java/src/java/org/apache/hadoop/thriftfs/ThriftHandlerBase.java

@@ -193,7 +193,9 @@ public abstract class ThriftHandlerBase implements HadoopServiceBase.Iface {
    */
   protected <T> T assumeUserContextAndExecute(RequestContext ctx, PrivilegedExceptionAction<T> action) throws IOException {
     try {
-      return UserGroupInformation.createProxyUser(ctx.confOptions.get("effective_user"), UserGroupInformation.getLoginUser()).doAs(action);
+      return UserGroupInformation.createProxyUser(
+        ctx.confOptions.get("effective_user"),
+        UserGroupInformation.getCurrentUser()).doAs(action);
     } catch (Throwable e) {
       throw ThriftUtils.toThrift(e);
     }
@@ -202,7 +204,7 @@ public abstract class ThriftHandlerBase implements HadoopServiceBase.Iface {
   protected <T> T assumeUserContextAndExecute(RequestContext ctx, PrivilegedAction<T> action) {
     try {
       return UserGroupInformation.createProxyUser(ctx.confOptions.get("effective_user"),
-          UserGroupInformation.getLoginUser()).doAs(action);
+          UserGroupInformation.getCurrentUser()).doAs(action);
     } catch (java.io.IOException e) {
       // This should only be thrown in the event getLoginUser() fails.
       throw new Error(e);

+ 18 - 3
desktop/libs/hadoop/java/src/java/org/apache/hadoop/thriftfs/ThriftPluginServer.java

@@ -26,10 +26,12 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.thrift.TProcessorFactory;
 import org.apache.thrift.protocol.TBinaryProtocol;
 import org.apache.thrift.transport.TServerSocket;
 import org.apache.thrift.transport.TServerTransport;
+import org.apache.thrift.transport.TTransportException;
 import org.apache.thrift.transport.TTransportFactory;
 
 /**
@@ -45,6 +47,8 @@ public class ThriftPluginServer implements Configurable, Runnable {
   private InetSocketAddress address;
 
   private TProcessorFactory processorFactory;
+  private TTransportFactory transportFactory;
+  private HadoopThriftAuthBridge.Server authBridge;
 
   static final Log LOG = LogFactory.getLog(ThriftPluginServer.class);
 
@@ -56,11 +60,22 @@ public class ThriftPluginServer implements Configurable, Runnable {
   }
 
   public ThriftPluginServer(InetSocketAddress address,
-                            TProcessorFactory processorFactory) {
+                            TProcessorFactory processorFactory)
+    throws TTransportException {
     //options = new TThreadPoolServer.Options();
     port = address.getPort();
     this.address = address;
-    this.processorFactory = processorFactory;
+
+    if (UserGroupInformation.isSecurityEnabled()) {
+      authBridge = new HadoopThriftAuthBridge.Server();
+
+      this.processorFactory = authBridge.wrapProcessorFactory(
+        processorFactory);
+      transportFactory = authBridge.createTransportFactory();
+    } else {
+      this.processorFactory = processorFactory;
+      transportFactory = new TTransportFactory();
+    }
   }
 
   /**
@@ -96,7 +111,7 @@ public class ThriftPluginServer implements Configurable, Runnable {
 
       server = new SanerThreadPoolServer(
         processorFactory, transport,
-        new TTransportFactory(), new TTransportFactory(),
+        transportFactory, transportFactory,
         new TBinaryProtocol.Factory(), new TBinaryProtocol.Factory(), options);
     }
 

+ 1 - 0
desktop/libs/hadoop/src/hadoop/cluster.py

@@ -38,6 +38,7 @@ def _make_filesystem(identifier):
       cluster_conf.NN_HOST.get(),
       cluster_conf.NN_THRIFT_PORT.get(),
       cluster_conf.NN_HDFS_PORT.get(),
+      kerberos_principal=cluster_conf.NN_KERBEROS_PRINCIPAL.get(),
       hadoop_bin_path=conf.HADOOP_BIN.get())
     raise Exception("Unknown choice: %s" % choice)
 

+ 3 - 1
desktop/libs/hadoop/src/hadoop/conf.py

@@ -114,7 +114,9 @@ HDFS_CLUSTERS = UnspecifiedConfigSection(
       NN_THRIFT_PORT=Config("thrift_port", help="Thrift port for name node", default=10090,
                             type=int),
       NN_HDFS_PORT=Config("hdfs_port", help="Hadoop IPC port for the name node", default=8020,
-                            type=int)
+                            type=int),
+      NN_KERBEROS_PRINCIPAL=Config("kerberos_principal", help="Kerberos principal for NameNode",
+                                   default="hdfs", type=str),
     )
   )
 )

+ 10 - 5
desktop/libs/hadoop/src/hadoop/fs/hadoopfs.py

@@ -93,10 +93,11 @@ def test_fs_configuration(fs_config, hadoop_bin_conf):
 
   # Check thrift plugin
   try:
-    fs = HadoopFileSystem(fs_config.NN_HOST.get(),
-                          fs_config.NN_THRIFT_PORT.get(),
-                          fs_config.NN_HDFS_PORT.get(),
-                          hadoop_bin_conf.get())
+    fs = HadoopFileSystem(host=fs_config.NN_HOST.get(),
+                          thrift_port=fs_config.NN_THRIFT_PORT.get(),
+                          hdfs_port=fs_config.NN_HDFS_PORT.get(),
+                          kerberos_principal=fs_config.NN_KERBEROS_PRINCIPAL.get(),
+                          hadoop_bin_path=hadoop_bin_conf.get())
 
     fs.setuser(fs.superuser)
     ls = fs.listdir('/')
@@ -176,7 +177,9 @@ class HadoopFileSystem(object):
   Implementation of Filesystem APIs through Thrift to a Hadoop cluster.
   """
 
-  def __init__(self, host, thrift_port, hdfs_port=8020, hadoop_bin_path="hadoop"):
+  def __init__(self, host, thrift_port, hdfs_port=8020,
+               kerberos_principal="hdfs",
+               hadoop_bin_path="hadoop"):
     """
     @param host hostname or IP of the namenode
     @param thrift_port port on which the Thrift plugin is listening
@@ -188,11 +191,13 @@ class HadoopFileSystem(object):
     self.host = host
     self.thrift_port = thrift_port
     self.hdfs_port = hdfs_port
+    self.kerberos_principal = kerberos_principal
     self.hadoop_bin_path = hadoop_bin_path
     self._resolve_hadoop_path()
 
     self.nn_client = thrift_util.get_client(Namenode.Client, host, thrift_port,
         service_name="HDFS Namenode HUE Plugin",
+        kerberos_principal=kerberos_principal,
         timeout_seconds=NN_THRIFT_TIMEOUT)
 
     # The file systems are cached globally.  We store