-
Notifications
You must be signed in to change notification settings - Fork 439
TEZ-4688 TEZ-4648: Tez upgrade to Hadoop 3.5.0 and jersey 2.x #474
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: master
Are you sure you want to change the base?
Changes from all commits
b28d2f7
fc9865d
fcd930a
a1e9c45
30c08bf
8f159f9
76a32cb
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -24,12 +24,13 @@ | |
| import java.lang.reflect.Method; | ||
| import java.net.HttpURLConnection; | ||
| import java.net.URL; | ||
| import java.net.URLEncoder; | ||
| import java.security.GeneralSecurityException; | ||
|
|
||
| import javax.net.ssl.HostnameVerifier; | ||
| import javax.net.ssl.HttpsURLConnection; | ||
| import javax.net.ssl.SSLSocketFactory; | ||
| import javax.ws.rs.client.Client; | ||
| import javax.ws.rs.client.ClientBuilder; | ||
|
|
||
| import org.apache.hadoop.classification.InterfaceAudience; | ||
| import org.apache.hadoop.conf.Configuration; | ||
|
|
@@ -42,13 +43,11 @@ | |
| import org.apache.tez.dag.api.TezException; | ||
|
|
||
| import com.google.common.annotations.VisibleForTesting; | ||
| import com.sun.jersey.api.client.Client; | ||
| import com.sun.jersey.api.client.config.ClientConfig; | ||
| import com.sun.jersey.api.client.config.DefaultClientConfig; | ||
| import com.sun.jersey.client.urlconnection.HttpURLConnectionFactory; | ||
| import com.sun.jersey.client.urlconnection.URLConnectionClientHandler; | ||
| import com.sun.jersey.json.impl.provider.entity.JSONRootElementProvider; | ||
|
|
||
| import org.glassfish.jersey.client.ClientConfig; | ||
| import org.glassfish.jersey.client.HttpUrlConnectorProvider; | ||
| import org.glassfish.jersey.client.HttpUrlConnectorProvider.ConnectionFactory; | ||
| import org.glassfish.jersey.jackson.JacksonFeature; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
|
|
@@ -161,10 +160,7 @@ public TimelineReaderTokenAuthenticatedStrategy(final Configuration conf, | |
| @Override | ||
| public Client getHttpClient() throws IOException { | ||
| Authenticator authenticator; | ||
| UserGroupInformation ugi = UserGroupInformation.getCurrentUser(); | ||
| UserGroupInformation realUgi = ugi.getRealUser(); | ||
| String doAsUser; | ||
| ClientConfig clientConfig = new DefaultClientConfig(JSONRootElementProvider.App.class); | ||
| ClientConfig clientConfig = new ClientConfig().register(JacksonFeature.class); | ||
| ConnectionConfigurator connectionConfigurator = getNewConnectionConf(useHttps, | ||
| connTimeout, sslFactory); | ||
|
|
||
|
|
@@ -175,20 +171,18 @@ public Client getHttpClient() throws IOException { | |
| throw new IOException("Failed to get authenticator", e); | ||
| } | ||
|
|
||
| if (realUgi != null) { | ||
| doAsUser = ugi.getShortUserName(); | ||
| } else { | ||
| doAsUser = null; | ||
| } | ||
| ConnectionFactory factory = url -> { | ||
| try { | ||
| return new AuthenticatedURL(authenticator) | ||
| .openConnection(url, new AuthenticatedURL.Token()); | ||
| } catch (Exception e) { | ||
| throw new IOException("Hadoop Authentication failed", e); | ||
| } | ||
| }; | ||
|
|
||
| HttpURLConnectionFactory connectionFactory; | ||
| try { | ||
| connectionFactory = new TokenAuthenticatedURLConnectionFactory(connectionConfigurator, authenticator, | ||
| doAsUser); | ||
| } catch (TezException e) { | ||
| throw new IOException("Fail to create TokenAuthenticatedURLConnectionFactory", e); | ||
| } | ||
| return new Client(new URLConnectionClientHandler(connectionFactory), clientConfig); | ||
| HttpUrlConnectorProvider connectorProvider = new HttpUrlConnectorProvider().connectionFactory(factory); | ||
| clientConfig.connectorProvider(connectorProvider); | ||
| return ClientBuilder.newClient(clientConfig); | ||
| } | ||
|
|
||
| private static Authenticator getTokenAuthenticator() throws TezException { | ||
|
|
@@ -203,42 +197,6 @@ private static Authenticator getTokenAuthenticator() throws TezException { | |
| return ReflectionUtils.createClazzInstance(authenticatorClazzName); | ||
| } | ||
|
|
||
| private static class TokenAuthenticatedURLConnectionFactory implements HttpURLConnectionFactory { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I checked and added Hadoop delegation token auth related checks in latest commit. Please check and let me know if any other scenarios we can cover. thanks ! |
||
|
|
||
| private final Authenticator authenticator; | ||
| private final ConnectionConfigurator connConfigurator; | ||
| private final String doAsUser; | ||
| private final AuthenticatedURL.Token token; | ||
|
|
||
| public TokenAuthenticatedURLConnectionFactory(ConnectionConfigurator connConfigurator, | ||
| Authenticator authenticator, | ||
| String doAsUser) throws TezException { | ||
| this.connConfigurator = connConfigurator; | ||
| this.authenticator = authenticator; | ||
| this.doAsUser = doAsUser; | ||
| this.token = ReflectionUtils.createClazzInstance( | ||
| DELEGATION_TOKEN_AUTHENTICATED_URL_TOKEN_CLASS_NAME, null, null); | ||
| } | ||
|
|
||
| @Override | ||
| public HttpURLConnection getHttpURLConnection(URL url) throws IOException { | ||
| try { | ||
| AuthenticatedURL authenticatedURL= ReflectionUtils.createClazzInstance( | ||
| DELEGATION_TOKEN_AUTHENTICATED_URL_CLAZZ_NAME, new Class[] { | ||
| delegationTokenAuthenticatorClazz, | ||
| ConnectionConfigurator.class | ||
| }, new Object[] { | ||
| authenticator, | ||
| connConfigurator | ||
| }); | ||
| return ReflectionUtils.invokeMethod(authenticatedURL, | ||
| delegationTokenAuthenticateURLOpenConnectionMethod, url, token, doAsUser); | ||
| } catch (Exception e) { | ||
| throw new IOException(e); | ||
| } | ||
| } | ||
| } | ||
|
|
||
| @Override | ||
| public void close() { | ||
| if (sslFactory != null) { | ||
|
|
@@ -265,31 +223,11 @@ public TimelineReaderPseudoAuthenticatedStrategy(final Configuration conf, | |
|
|
||
| @Override | ||
| public Client getHttpClient() { | ||
| ClientConfig config = new DefaultClientConfig(JSONRootElementProvider.App.class); | ||
| HttpURLConnectionFactory urlFactory = new PseudoAuthenticatedURLConnectionFactory(connectionConf); | ||
| return new Client(new URLConnectionClientHandler(urlFactory), config); | ||
| ClientConfig config = new ClientConfig().register(JacksonFeature.class); | ||
| return ClientBuilder.newClient(config); | ||
| } | ||
|
|
||
| @VisibleForTesting | ||
| protected static class PseudoAuthenticatedURLConnectionFactory implements HttpURLConnectionFactory { | ||
| private final ConnectionConfigurator connectionConf; | ||
|
|
||
| public PseudoAuthenticatedURLConnectionFactory(ConnectionConfigurator connectionConf) { | ||
| this.connectionConf = connectionConf; | ||
| } | ||
|
|
||
| @Override | ||
| public HttpURLConnection getHttpURLConnection(URL url) throws IOException { | ||
| String tokenString = (url.getQuery() == null ? "?" : "&") + "user.name=" + | ||
| URLEncoder.encode(UserGroupInformation.getCurrentUser().getShortUserName(), "UTF8"); | ||
|
|
||
| HttpURLConnection httpURLConnection = | ||
| (HttpURLConnection) (new URL(url + tokenString)).openConnection(); | ||
| this.connectionConf.configure(httpURLConnection); | ||
|
|
||
| return httpURLConnection; | ||
| } | ||
| } | ||
| // PseudoAuthenticatedURLConnectionFactory removed in Jersey 2 migration | ||
|
|
||
| @Override | ||
| public void close() { | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.