in kyuubi-relocated-hive-metastore-client/src/main/java/org/apache/kyuubi/shaded/hive/metastore/HiveMetaStoreClient.java [569:673]
private TTransport createAuthBinaryTransport(URI store, TTransport underlyingTransport)
throws MetaException {
boolean isHttpTransportMode =
MetastoreConf.getVar(conf, ConfVars.METASTORE_CLIENT_THRIFT_TRANSPORT_MODE)
.equalsIgnoreCase("http");
if (isHttpTransportMode) {
throw new IllegalArgumentException("HTTP mode is not supported");
}
Objects.requireNonNull(underlyingTransport, "Underlying transport should not be null");
// default transport is the underlying one
TTransport transport = underlyingTransport;
boolean useFramedTransport =
MetastoreConf.getBoolVar(conf, ConfVars.USE_THRIFT_FRAMED_TRANSPORT);
boolean useSSL = MetastoreConf.getBoolVar(conf, ConfVars.USE_SSL);
boolean useSasl = MetastoreConf.getBoolVar(conf, ConfVars.USE_THRIFT_SASL);
String clientAuthMode = MetastoreConf.getVar(conf, ConfVars.METASTORE_CLIENT_AUTH_MODE);
boolean usePasswordAuth = false;
if (clientAuthMode != null) {
usePasswordAuth = "PLAIN".equalsIgnoreCase(clientAuthMode);
}
if (usePasswordAuth) {
// we are using PLAIN Sasl connection with user/password
LOG.debug("HMSC::open(): Creating plain authentication thrift connection.");
String userName = MetastoreConf.getVar(conf, ConfVars.METASTORE_CLIENT_PLAIN_USERNAME);
if (null == userName || userName.isEmpty()) {
throw new MetaException("No user specified for plain transport.");
}
// The password is not directly provided. It should be obtained from a keystore pointed
// by configuration "hadoop.security.credential.provider.path".
try {
String passwd = null;
char[] pwdCharArray = conf.getPassword(userName);
if (null != pwdCharArray) {
passwd = new String(pwdCharArray);
}
if (null == passwd) {
throw new MetaException("No password found for user " + userName);
}
// Overlay the SASL transport on top of the base socket transport (SSL or non-SSL)
transport =
MetaStorePlainSaslHelper.getPlainTransport(userName, passwd, underlyingTransport);
} catch (IOException | TTransportException sasle) {
// IOException covers SaslException
LOG.error("Could not create client transport", sasle);
throw new MetaException(sasle.toString());
}
} else if (useSasl) {
// Wrap thrift connection with SASL for secure connection.
try {
HadoopThriftAuthBridge.Client authBridge =
HadoopThriftAuthBridge.getBridge().createClient();
// check if we should use delegation tokens to authenticate
// the call below gets hold of the tokens if they are set up by hadoop
// this should happen on the map/reduce tasks if the client added the
// tokens into hadoop's credential store in the front end during job
// submission.
String tokenSig = MetastoreConf.getVar(conf, ConfVars.TOKEN_SIGNATURE);
// tokenSig could be null
tokenStrForm = SecurityUtils.getTokenStrForm(tokenSig);
if (tokenStrForm != null) {
LOG.debug(
"HMSC::open(): Found delegation token. Creating DIGEST-based thrift connection.");
// authenticate using delegation tokens via the "DIGEST" mechanism
transport =
authBridge.createClientTransport(
null,
store.getHost(),
"DIGEST",
tokenStrForm,
underlyingTransport,
MetaStoreUtils.getMetaStoreSaslProperties(conf, useSSL));
} else {
LOG.debug(
"HMSC::open(): Could not find delegation token. Creating KERBEROS-based thrift connection.");
String principalConfig = MetastoreConf.getVar(conf, ConfVars.KERBEROS_PRINCIPAL);
transport =
authBridge.createClientTransport(
principalConfig,
store.getHost(),
"KERBEROS",
null,
underlyingTransport,
MetaStoreUtils.getMetaStoreSaslProperties(conf, useSSL));
}
} catch (IOException ioe) {
LOG.error("Failed to create client transport", ioe);
throw new MetaException(ioe.toString());
}
} else {
if (useFramedTransport) {
try {
transport = new TFramedTransport(transport);
} catch (TTransportException e) {
LOG.error("Failed to create client transport", e);
throw new MetaException(e.toString());
}
}
}
return transport;
}