0001
0002
0003
0004
0005
0006
0007
0008
0009
0010
0011
0012
0013
0014
0015
0016
0017
0018 package org.apache.hive.service.auth;
0019
0020 import java.io.IOException;
0021 import java.util.Map;
0022 import javax.security.sasl.SaslException;
0023
0024 import org.apache.hadoop.hive.shims.ShimLoader;
0025 import org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge;
0026 import org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge.Server;
0027 import org.apache.hive.service.cli.thrift.ThriftCLIService;
0028 import org.apache.hive.service.rpc.thrift.TCLIService;
0029 import org.apache.hive.service.rpc.thrift.TCLIService.Iface;
0030 import org.apache.thrift.TProcessor;
0031 import org.apache.thrift.TProcessorFactory;
0032 import org.apache.thrift.transport.TSaslClientTransport;
0033 import org.apache.thrift.transport.TTransport;
0034
0035 public final class KerberosSaslHelper {
0036
0037 public static TProcessorFactory getKerberosProcessorFactory(Server saslServer,
0038 ThriftCLIService service) {
0039 return new CLIServiceProcessorFactory(saslServer, service);
0040 }
0041
0042 public static TTransport getKerberosTransport(String principal, String host,
0043 TTransport underlyingTransport, Map<String, String> saslProps, boolean assumeSubject)
0044 throws SaslException {
0045 try {
0046 String[] names = principal.split("[/@]");
0047 if (names.length != 3) {
0048 throw new IllegalArgumentException("Kerberos principal should have 3 parts: " + principal);
0049 }
0050
0051 if (assumeSubject) {
0052 return createSubjectAssumedTransport(principal, underlyingTransport, saslProps);
0053 } else {
0054 HadoopThriftAuthBridge.Client authBridge =
0055 ShimLoader.getHadoopThriftAuthBridge().createClientWithConf("kerberos");
0056 return authBridge.createClientTransport(principal, host, "KERBEROS", null,
0057 underlyingTransport, saslProps);
0058 }
0059 } catch (IOException e) {
0060 throw new SaslException("Failed to open client transport", e);
0061 }
0062 }
0063
0064 public static TTransport createSubjectAssumedTransport(String principal,
0065 TTransport underlyingTransport, Map<String, String> saslProps) throws IOException {
0066 String[] names = principal.split("[/@]");
0067 try {
0068 TTransport saslTransport =
0069 new TSaslClientTransport("GSSAPI", null, names[0], names[1], saslProps, null,
0070 underlyingTransport);
0071 return new TSubjectAssumingTransport(saslTransport);
0072 } catch (SaslException se) {
0073 throw new IOException("Could not instantiate SASL transport", se);
0074 }
0075 }
0076
0077 public static TTransport getTokenTransport(String tokenStr, String host,
0078 TTransport underlyingTransport, Map<String, String> saslProps) throws SaslException {
0079 HadoopThriftAuthBridge.Client authBridge =
0080 ShimLoader.getHadoopThriftAuthBridge().createClientWithConf("kerberos");
0081
0082 try {
0083 return authBridge.createClientTransport(null, host, "DIGEST", tokenStr, underlyingTransport,
0084 saslProps);
0085 } catch (IOException e) {
0086 throw new SaslException("Failed to open client transport", e);
0087 }
0088 }
0089
0090 private KerberosSaslHelper() {
0091 throw new UnsupportedOperationException("Can't initialize class");
0092 }
0093
0094 private static class CLIServiceProcessorFactory extends TProcessorFactory {
0095
0096 private final ThriftCLIService service;
0097 private final Server saslServer;
0098
0099 CLIServiceProcessorFactory(Server saslServer, ThriftCLIService service) {
0100 super(null);
0101 this.service = service;
0102 this.saslServer = saslServer;
0103 }
0104
0105 @Override
0106 public TProcessor getProcessor(TTransport trans) {
0107 TProcessor sqlProcessor = new TCLIService.Processor<Iface>(service);
0108 return saslServer.wrapNonAssumingProcessor(sqlProcessor);
0109 }
0110 }
0111 }