-
Notifications
You must be signed in to change notification settings - Fork 441
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
d5cc9ef
eab1b7d
af2f4db
f9f1cad
e548565
044f792
d37d8cd
1a01f07
43765a7
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 |
|---|---|---|
|
|
@@ -30,25 +30,23 @@ | |
| 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; | ||
| import org.apache.hadoop.security.UserGroupInformation; | ||
| import org.apache.hadoop.security.authentication.client.AuthenticatedURL; | ||
| import org.apache.hadoop.security.authentication.client.Authenticator; | ||
| import org.apache.hadoop.security.authentication.client.ConnectionConfigurator; | ||
| import org.apache.hadoop.security.ssl.SSLFactory; | ||
| import org.apache.tez.common.ReflectionUtils; | ||
| 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.jackson.JacksonFeature; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
|
|
@@ -164,7 +162,7 @@ public Client getHttpClient() throws IOException { | |
| 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); | ||
|
|
||
|
|
@@ -181,14 +179,16 @@ public Client getHttpClient() throws IOException { | |
| doAsUser = null; | ||
| } | ||
|
|
||
| HttpURLConnectionFactory connectionFactory; | ||
| HttpUrlConnectorProvider.ConnectionFactory 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); | ||
|
|
||
| clientConfig.connectorProvider(new HttpUrlConnectorProvider().connectionFactory(connectionFactory)); | ||
| return ClientBuilder.newClient(clientConfig); | ||
| } | ||
|
|
||
| private static Authenticator getTokenAuthenticator() throws TezException { | ||
|
|
@@ -203,12 +203,12 @@ 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 !
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. I haven't checked this one in detail yet, please let us know @Aggarwal-Raghav if you're fine with @maheshrajus's analysis, so we can maybe move forward
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. @maheshrajus , I'm not in favour of removing To summarize, in this PR you are creating a client without plugging Can you check this patch on top your PR?
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. Fixed. @Aggarwal-Raghav Can you please check and confirm. Thanks ! |
||
| private static class TokenAuthenticatedURLConnectionFactory implements HttpUrlConnectorProvider.ConnectionFactory { | ||
|
|
||
| private final Authenticator authenticator; | ||
| private final ConnectionConfigurator connConfigurator; | ||
| private final String doAsUser; | ||
| private final AuthenticatedURL.Token token; | ||
| private final Object token; | ||
|
|
||
| public TokenAuthenticatedURLConnectionFactory(ConnectionConfigurator connConfigurator, | ||
| Authenticator authenticator, | ||
|
|
@@ -221,17 +221,17 @@ public TokenAuthenticatedURLConnectionFactory(ConnectionConfigurator connConfigu | |
| } | ||
|
|
||
| @Override | ||
| public HttpURLConnection getHttpURLConnection(URL url) throws IOException { | ||
| public HttpURLConnection getConnection(URL url) throws IOException { | ||
| try { | ||
| AuthenticatedURL authenticatedURL= ReflectionUtils.createClazzInstance( | ||
| Object authenticatedURL = ReflectionUtils.createClazzInstance( | ||
| DELEGATION_TOKEN_AUTHENTICATED_URL_CLAZZ_NAME, new Class[] { | ||
| delegationTokenAuthenticatorClazz, | ||
| ConnectionConfigurator.class | ||
| }, new Object[] { | ||
| authenticator, | ||
| connConfigurator | ||
| }); | ||
| return ReflectionUtils.invokeMethod(authenticatedURL, | ||
| return (HttpURLConnection) ReflectionUtils.invokeMethod(authenticatedURL, | ||
| delegationTokenAuthenticateURLOpenConnectionMethod, url, token, doAsUser); | ||
| } catch (Exception e) { | ||
| throw new IOException(e); | ||
|
|
@@ -265,21 +265,24 @@ 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); | ||
| HttpUrlConnectorProvider.ConnectionFactory urlFactory = | ||
| new PseudoAuthenticatedURLConnectionFactory(connectionConf); | ||
| config.connectorProvider(new HttpUrlConnectorProvider().connectionFactory(urlFactory)); | ||
| return ClientBuilder.newClient(config); | ||
| } | ||
|
|
||
| @VisibleForTesting | ||
| protected static class PseudoAuthenticatedURLConnectionFactory implements HttpURLConnectionFactory { | ||
| protected static class PseudoAuthenticatedURLConnectionFactory | ||
| implements HttpUrlConnectorProvider.ConnectionFactory { | ||
| private final ConnectionConfigurator connectionConf; | ||
|
|
||
| public PseudoAuthenticatedURLConnectionFactory(ConnectionConfigurator connectionConf) { | ||
| this.connectionConf = connectionConf; | ||
| } | ||
|
|
||
| @Override | ||
| public HttpURLConnection getHttpURLConnection(URL url) throws IOException { | ||
| public HttpURLConnection getConnection(URL url) throws IOException { | ||
| String tokenString = (url.getQuery() == null ? "?" : "&") + "user.name=" + | ||
| URLEncoder.encode(UserGroupInformation.getCurrentUser().getShortUserName(), "UTF8"); | ||
|
|
||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.