From 6e04605afdddbba7baa35c13a1867a2b56811d4d Mon Sep 17 00:00:00 2001 From: Piotr Stankowski Date: Thu, 2 Mar 2023 07:25:28 +0100 Subject: [PATCH 1/7] WebHdfs: add support for basic auth and custom API path --- .../hdfs/client/HdfsClientConfigKeys.java | 3 + .../hdfs/web/BasicAuthConfigurator.java | 67 +++++++++++++++++ .../hadoop/hdfs/web/URLConnectionFactory.java | 39 +++++++++- .../hadoop/hdfs/web/WebHdfsFileSystem.java | 25 ++++++- .../hdfs/web/TestBasicAuthConfigurator.java | 67 +++++++++++++++++ .../src/main/resources/hdfs-default.xml | 8 ++ .../hadoop/hdfs/web/TestWebHdfsUrl.java | 73 ++++++++++++++++++- 7 files changed, 273 insertions(+), 9 deletions(-) create mode 100644 hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/BasicAuthConfigurator.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/web/TestBasicAuthConfigurator.java diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java index 2b511bfc2ebee..cd61e411043e3 100755 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java @@ -287,6 +287,9 @@ public interface HdfsClientConfigKeys { "dfs.client.output.stream.uniq.default.key"; String DFS_OUTPUT_STREAM_UNIQ_DEFAULT_KEY_DEFAULT = "DEFAULT"; + String DFS_CLIENT_WEBHDFS_USE_BASE_PATH_KEY = "dfs.client.webhdfs.use-base-path"; + boolean DFS_CLIENT_WEBHDFS_USE_BASE_PATH_DEFAULT = false; + /** * These are deprecated config keys to client code. */ diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/BasicAuthConfigurator.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/BasicAuthConfigurator.java new file mode 100644 index 0000000000000..0a8809c980a02 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/BasicAuthConfigurator.java @@ -0,0 +1,67 @@ +/** + * 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.hdfs.web; + +import org.apache.hadoop.security.authentication.client.ConnectionConfigurator; + +import java.io.IOException; +import java.net.HttpURLConnection; +import java.util.Base64; + +/** + * This class adds basic authentication to the connection, + * allowing users to access webhdfs over HTTP with basic authentication, + * for example when using Apache Knox. + */ +public class BasicAuthConfigurator implements ConnectionConfigurator { + private final ConnectionConfigurator parent; + private final String credentials; + + /** + * @param credentials a string of the form "username:password" + */ + public BasicAuthConfigurator( + ConnectionConfigurator parent, + String credentials + ) { + this.parent = parent; + this.credentials = credentials; + } + + @Override + public HttpURLConnection configure(HttpURLConnection conn) throws IOException { + if (parent != null) { + parent.configure(conn); + } + + if (credentials != null && !credentials.equals("")) { + conn.setRequestProperty( + "AUTHORIZATION", + "Basic " + Base64.getEncoder().encodeToString(credentials.getBytes()) + ); + } + + return conn; + } + + public void destroy() { + if (parent != null && parent instanceof SSLConnectionConfigurator) { + ((SSLConnectionConfigurator)parent).destroy(); + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java index 9cfe3b6a0447f..da441f6127e5d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java @@ -77,7 +77,7 @@ public HttpURLConnection configure(HttpURLConnection conn) public static URLConnectionFactory newDefaultURLConnectionFactory( Configuration conf) { ConnectionConfigurator conn = getSSLConnectionConfiguration( - DEFAULT_SOCKET_TIMEOUT, DEFAULT_SOCKET_TIMEOUT, conf); + DEFAULT_SOCKET_TIMEOUT, DEFAULT_SOCKET_TIMEOUT, conf, null); return new URLConnectionFactory(conn); } @@ -89,12 +89,28 @@ public static URLConnectionFactory newDefaultURLConnectionFactory( public static URLConnectionFactory newDefaultURLConnectionFactory( int connectTimeout, int readTimeout, Configuration conf) { ConnectionConfigurator conn = getSSLConnectionConfiguration( - connectTimeout, readTimeout, conf); + connectTimeout, readTimeout, conf, null); + return new URLConnectionFactory(conn); + } + + /** + * Construct a new URLConnectionFactory based on the configuration. It will + * honor connecTimeout and readTimeout when they are specified and allows + * specifying credentials for HTTP Basic Authentication in "username:password" format. + */ + public static URLConnectionFactory newDefaultURLConnectionFactory( + int connectTimeout, int readTimeout, Configuration conf, String basicAuthCredentials) { + ConnectionConfigurator conn = getSSLConnectionConfiguration( + connectTimeout, readTimeout, conf, basicAuthCredentials); return new URLConnectionFactory(conn); } private static ConnectionConfigurator getSSLConnectionConfiguration( - final int connectTimeout, final int readTimeout, Configuration conf) { + final int connectTimeout, + final int readTimeout, + Configuration conf, + String basicAuthCredentials + ) { ConnectionConfigurator conn; try { conn = new SSLConnectionConfigurator(connectTimeout, readTimeout, conf); @@ -120,7 +136,7 @@ public HttpURLConnection configure(HttpURLConnection connection) } } - return conn; + return new BasicAuthConfigurator(conn, basicAuthCredentials); } /** @@ -130,6 +146,17 @@ public HttpURLConnection configure(HttpURLConnection connection) public static URLConnectionFactory newOAuth2URLConnectionFactory( int connectTimeout, int readTimeout, Configuration conf) throws IOException { + return newOAuth2URLConnectionFactory(connectTimeout, readTimeout, conf, null); + } + + /** + * Construct a new URLConnectionFactory that supports OAuth-based connections. + * It will also try to load the SSL configuration when they are specified. Furthermore, it allows + * specifying credentials for HTTP Basic Authentication in "username:password" format. + */ + public static URLConnectionFactory newOAuth2URLConnectionFactory( + int connectTimeout, int readTimeout, Configuration conf, String basicAuthCredentials) + throws IOException { ConnectionConfigurator conn; try { ConnectionConfigurator sslConnConfigurator @@ -139,6 +166,7 @@ public static URLConnectionFactory newOAuth2URLConnectionFactory( } catch (Exception e) { throw new IOException("Unable to load OAuth2 connection factory.", e); } + conn = new BasicAuthConfigurator(conn, basicAuthCredentials); return new URLConnectionFactory(conn); } @@ -214,5 +242,8 @@ public void destroy() { if (connConfigurator instanceof SSLConnectionConfigurator) { ((SSLConnectionConfigurator) connConfigurator).destroy(); } + if (connConfigurator instanceof BasicAuthConfigurator) { + ((BasicAuthConfigurator) connConfigurator).destroy(); + } } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java index f0774e98d1f8d..b71bc84e0bca1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java @@ -184,6 +184,7 @@ public class WebHdfsFileSystem extends FileSystem private boolean isTLSKrb; private boolean isServerHCFSCompatible = true; + private String pathPrefix = ""; /** * Return the protocol scheme for the FileSystem. @@ -238,17 +239,19 @@ public synchronized void initialize(URI uri, Configuration conf if(isOAuth) { LOG.debug("Enabling OAuth2 in WebHDFS"); connectionFactory = URLConnectionFactory - .newOAuth2URLConnectionFactory(connectTimeout, readTimeout, conf); + .newOAuth2URLConnectionFactory(connectTimeout, readTimeout, conf, uri.getUserInfo()); } else { LOG.debug("Not enabling OAuth2 in WebHDFS"); connectionFactory = URLConnectionFactory - .newDefaultURLConnectionFactory(connectTimeout, readTimeout, conf); + .newDefaultURLConnectionFactory(connectTimeout, readTimeout, conf, uri.getUserInfo()); } this.isTLSKrb = "HTTPS_ONLY".equals(conf.get(DFS_HTTP_POLICY_KEY)); ugi = UserGroupInformation.getCurrentUser(); - this.uri = URI.create(uri.getScheme() + "://" + uri.getAuthority()); + // Drop path and user:password from URI. + this.uri = URI.create(uri.getScheme() + "://" + uri.getHost() + + (uri.getPort() == -1 ? "" : ":" + uri.getPort())); this.nnAddrs = resolveNNAddr(); boolean isHA = HAUtilClient.isClientFailoverConfigured(conf, this.uri); @@ -307,6 +310,20 @@ public StorageStatistics provide() { return new DFSOpsCountStatistics(); } }); + pathPrefix = PATH_PREFIX; + boolean useBasePath = conf.getBoolean( + HdfsClientConfigKeys.DFS_CLIENT_WEBHDFS_USE_BASE_PATH_KEY, + HdfsClientConfigKeys.DFS_CLIENT_WEBHDFS_USE_BASE_PATH_DEFAULT + ); + if (uri.getPath() != null && !uri.getPath().equals("") && useBasePath) { + pathPrefix = uri.getPath(); + if (pathPrefix.endsWith("/")) { + pathPrefix = pathPrefix.substring(0, pathPrefix.length() - 1); + } + if (!pathPrefix.endsWith(PATH_PREFIX)) { + pathPrefix += PATH_PREFIX; + } + } } /** @@ -634,7 +651,7 @@ URL toUrl(final HttpOpParam.Op op, final Path fspath, final Param... parameters) throws IOException { //initialize URI path and query - final String path = PATH_PREFIX + final String path = pathPrefix + (fspath == null? "/": makeQualified(fspath).toUri().getRawPath()); final String query = op.toQueryString() + Param.toSortedString("&", getAuthParameters(op)) diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/web/TestBasicAuthConfigurator.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/web/TestBasicAuthConfigurator.java new file mode 100644 index 0000000000000..16ed04ef4dacb --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/web/TestBasicAuthConfigurator.java @@ -0,0 +1,67 @@ +/** + * 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.hdfs.web; + +import org.apache.hadoop.security.authentication.client.ConnectionConfigurator; +import org.junit.Test; +import org.mockito.Mockito; + +import java.io.IOException; +import java.net.HttpURLConnection; + +public class TestBasicAuthConfigurator { + @Test + public void testNullCredentials() throws IOException { + ConnectionConfigurator conf = new BasicAuthConfigurator(null, null); + HttpURLConnection conn = Mockito.mock(HttpURLConnection.class); + conf.configure(conn); + Mockito.verify(conn, Mockito.never()).setRequestProperty(Mockito.any(), Mockito.any()); + } + + @Test + public void testEmptyCredentials() throws IOException { + ConnectionConfigurator conf = new BasicAuthConfigurator(null, ""); + HttpURLConnection conn = Mockito.mock(HttpURLConnection.class); + conf.configure(conn); + Mockito.verify(conn, Mockito.never()).setRequestProperty(Mockito.any(), Mockito.any()); + } + + @Test + public void testCredentialsSet() throws IOException { + ConnectionConfigurator conf = new BasicAuthConfigurator(null, "user:pass"); + HttpURLConnection conn = Mockito.mock(HttpURLConnection.class); + conf.configure(conn); + Mockito.verify(conn, Mockito.times(1)).setRequestProperty( + "AUTHORIZATION", + "Basic dXNlcjpwYXNz" + ); + } + + @Test + public void testParentConfigurator() throws IOException { + ConnectionConfigurator parent = Mockito.mock(ConnectionConfigurator.class); + ConnectionConfigurator conf = new BasicAuthConfigurator(parent, "user:pass"); + HttpURLConnection conn = Mockito.mock(HttpURLConnection.class); + conf.configure(conn); + Mockito.verify(conn, Mockito.times(1)).setRequestProperty( + "AUTHORIZATION", + "Basic dXNlcjpwYXNz" + ); + Mockito.verify(parent, Mockito.times(1)).configure(conn); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml index 5643a9b5c5ee1..eecd5cefd65a9 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml @@ -6472,4 +6472,12 @@ Enables observer reads for clients. This should only be enabled when clients are using routers. + + dfs.client.webhdfs.use-base-path + false + + Enables webhdfs FS clients to use specified filesystem URL path as the prefix for API endpoint. + Useful when using a proxy server like Apache Knox, which requires prefixing requests with /gateway/name. + + diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsUrl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsUrl.java index cb62288096396..187329782f33a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsUrl.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsUrl.java @@ -373,7 +373,7 @@ private void checkQueryParams(String[] expected, URL url) { } private WebHdfsFileSystem getWebHdfsFileSystem(UserGroupInformation ugi, - Configuration conf) throws IOException { + Configuration conf, URI uri) throws IOException { if (UserGroupInformation.isSecurityEnabled()) { DelegationTokenIdentifier dtId = new DelegationTokenIdentifier(new Text( ugi.getUserName()), null, null); @@ -391,6 +391,11 @@ private WebHdfsFileSystem getWebHdfsFileSystem(UserGroupInformation ugi, return (WebHdfsFileSystem) FileSystem.get(uri, conf); } + private WebHdfsFileSystem getWebHdfsFileSystem(UserGroupInformation ugi, + Configuration conf) throws IOException { + return getWebHdfsFileSystem(ugi, conf, uri); + } + private static final String SPECIAL_CHARACTER_FILENAME = "specialFile ?\"\\()[]_-=&+;,{}#%'`~!@$^*|<>."; @@ -534,4 +539,70 @@ public void testWebHdfsPathWithSemicolon() throws Exception { dfs.getFileStatus(percent).getPath().getName()); } } + + @Test(timeout=60000) + public void testPathInUrlIgnored() throws IOException, URISyntaxException { + Configuration conf = new Configuration(); + + UserGroupInformation ugi = + UserGroupInformation.createRemoteUser("test-user"); + UserGroupInformation.setLoginUser(ugi); + + WebHdfsFileSystem webhdfs = getWebHdfsFileSystem( + ugi, conf, new URI("webhdfs://localhost/test") + ); + Path fsPath = new Path("/test-file"); + + URL fileStatusUrl = webhdfs.toUrl(GetOpParam.Op.GETFILESTATUS, fsPath); + assertEquals("/webhdfs/v1/test-file", fileStatusUrl.getPath()); + } + + @Test(timeout=60000) + public void testUseBasePathEnabledWithoutPath() throws IOException { + Configuration conf = new Configuration(); + conf.setBoolean("dfs.client.webhdfs.use-base-path", true); + + UserGroupInformation ugi = UserGroupInformation.createRemoteUser("test-user"); + UserGroupInformation.setLoginUser(ugi); + + WebHdfsFileSystem webhdfs = getWebHdfsFileSystem(ugi, conf); + Path fsPath = new Path("/test-file"); + + URL fileStatusUrl = webhdfs.toUrl(GetOpParam.Op.GETFILESTATUS, fsPath); + assertEquals("/webhdfs/v1/test-file", fileStatusUrl.getPath()); + } + + @Test(timeout=60000) + public void testUseBasePathEnabled() throws IOException, URISyntaxException { + Configuration conf = new Configuration(); + conf.setBoolean("dfs.client.webhdfs.use-base-path", true); + + UserGroupInformation ugi = UserGroupInformation.createRemoteUser("test-user"); + UserGroupInformation.setLoginUser(ugi); + + WebHdfsFileSystem webhdfs = getWebHdfsFileSystem( + ugi, conf, new URI("webhdfs://localhost/base-path/test/") + ); + Path fsPath = new Path("/test-file"); + + URL fileStatusUrl = webhdfs.toUrl(GetOpParam.Op.GETFILESTATUS, fsPath); + assertEquals("/base-path/test/webhdfs/v1/test-file", fileStatusUrl.getPath()); + } + + @Test(timeout=60000) + public void testApiPrefixInBasePathIgnored() throws IOException, URISyntaxException { + Configuration conf = new Configuration(); + conf.setBoolean("dfs.client.webhdfs.use-base-path", true); + + UserGroupInformation ugi = UserGroupInformation.createRemoteUser("test-user"); + UserGroupInformation.setLoginUser(ugi); + + WebHdfsFileSystem webhdfs = getWebHdfsFileSystem( + ugi, conf, new URI("webhdfs://localhost/base-path/test/webhdfs/v1") + ); + Path fsPath = new Path("/test-file"); + + URL fileStatusUrl = webhdfs.toUrl(GetOpParam.Op.GETFILESTATUS, fsPath); + assertEquals("/base-path/test/webhdfs/v1/test-file", fileStatusUrl.getPath()); + } } From 3febb1ccfec1269cbac3661c9716867f6f5905be Mon Sep 17 00:00:00 2001 From: Piotr Stankowski Date: Thu, 2 Mar 2023 21:13:31 +0100 Subject: [PATCH 2/7] WebHdfs: use UTF-8 for basic auth --- .../java/org/apache/hadoop/hdfs/web/BasicAuthConfigurator.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/BasicAuthConfigurator.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/BasicAuthConfigurator.java index 0a8809c980a02..0a56e9f01fa99 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/BasicAuthConfigurator.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/BasicAuthConfigurator.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.net.HttpURLConnection; +import java.nio.charset.StandardCharsets; import java.util.Base64; /** @@ -52,7 +53,7 @@ public HttpURLConnection configure(HttpURLConnection conn) throws IOException { if (credentials != null && !credentials.equals("")) { conn.setRequestProperty( "AUTHORIZATION", - "Basic " + Base64.getEncoder().encodeToString(credentials.getBytes()) + "Basic " + Base64.getEncoder().encodeToString(credentials.getBytes(StandardCharsets.UTF_8)) ); } From ee843013b36a761d3050c69e008f6bd234ae6985 Mon Sep 17 00:00:00 2001 From: Piotr Stankowski Date: Thu, 2 Mar 2023 21:14:59 +0100 Subject: [PATCH 3/7] WebHdfs: avoid hiding uri field in test --- .../java/org/apache/hadoop/hdfs/web/TestWebHdfsUrl.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsUrl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsUrl.java index 187329782f33a..f4be76d1449fe 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsUrl.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsUrl.java @@ -373,7 +373,7 @@ private void checkQueryParams(String[] expected, URL url) { } private WebHdfsFileSystem getWebHdfsFileSystem(UserGroupInformation ugi, - Configuration conf, URI uri) throws IOException { + Configuration conf, URI fsUri) throws IOException { if (UserGroupInformation.isSecurityEnabled()) { DelegationTokenIdentifier dtId = new DelegationTokenIdentifier(new Text( ugi.getUserName()), null, null); @@ -384,11 +384,11 @@ private WebHdfsFileSystem getWebHdfsFileSystem(UserGroupInformation ugi, Token token = new Token( dtId, dtSecretManager); SecurityUtil.setTokenService( - token, NetUtils.createSocketAddr(uri.getAuthority())); + token, NetUtils.createSocketAddr(fsUri.getAuthority())); token.setKind(WebHdfsConstants.WEBHDFS_TOKEN_KIND); ugi.addToken(token); } - return (WebHdfsFileSystem) FileSystem.get(uri, conf); + return (WebHdfsFileSystem) FileSystem.get(fsUri, conf); } private WebHdfsFileSystem getWebHdfsFileSystem(UserGroupInformation ugi, From 5babfbce1916a11567382920024a43026c82eb04 Mon Sep 17 00:00:00 2001 From: Piotr Stankowski Date: Fri, 3 Mar 2023 04:58:21 +0100 Subject: [PATCH 4/7] WebHdfs: reformat BasicAuthConfigurator --- .../org/apache/hadoop/hdfs/web/BasicAuthConfigurator.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/BasicAuthConfigurator.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/BasicAuthConfigurator.java index 0a56e9f01fa99..cb4879ca0a783 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/BasicAuthConfigurator.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/BasicAuthConfigurator.java @@ -53,7 +53,9 @@ public HttpURLConnection configure(HttpURLConnection conn) throws IOException { if (credentials != null && !credentials.equals("")) { conn.setRequestProperty( "AUTHORIZATION", - "Basic " + Base64.getEncoder().encodeToString(credentials.getBytes(StandardCharsets.UTF_8)) + "Basic " + Base64.getEncoder().encodeToString( + credentials.getBytes(StandardCharsets.UTF_8) + ) ); } From 87e39011908cd63627beeab0b68ef89a2ac72454 Mon Sep 17 00:00:00 2001 From: Piotr Stankowski Date: Sat, 11 Mar 2023 14:08:16 -0500 Subject: [PATCH 5/7] WebHdfs: tweak tests and BasicAuthConfigurator instantiation --- .../hdfs/web/BasicAuthConfigurator.java | 10 ++++++++ .../hadoop/hdfs/web/URLConnectionFactory.java | 4 ++-- .../hdfs/web/TestBasicAuthConfigurator.java | 24 +++++++++++++++---- 3 files changed, 32 insertions(+), 6 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/BasicAuthConfigurator.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/BasicAuthConfigurator.java index cb4879ca0a783..67dcd2b861120 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/BasicAuthConfigurator.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/BasicAuthConfigurator.java @@ -33,6 +33,16 @@ public class BasicAuthConfigurator implements ConnectionConfigurator { private final ConnectionConfigurator parent; private final String credentials; + static public ConnectionConfigurator getConfigurator( + ConnectionConfigurator configurator, String basicAuthCred + ) { + if (basicAuthCred != null && !basicAuthCred.isEmpty()) { + return new BasicAuthConfigurator(configurator, basicAuthCred); + } else { + return configurator; + } + } + /** * @param credentials a string of the form "username:password" */ diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java index da441f6127e5d..9913b5a0d0018 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java @@ -136,7 +136,7 @@ public HttpURLConnection configure(HttpURLConnection connection) } } - return new BasicAuthConfigurator(conn, basicAuthCredentials); + return BasicAuthConfigurator.getConfigurator(conn, basicAuthCredentials); } /** @@ -166,7 +166,7 @@ public static URLConnectionFactory newOAuth2URLConnectionFactory( } catch (Exception e) { throw new IOException("Unable to load OAuth2 connection factory.", e); } - conn = new BasicAuthConfigurator(conn, basicAuthCredentials); + conn = BasicAuthConfigurator.getConfigurator(conn, basicAuthCredentials); return new URLConnectionFactory(conn); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/web/TestBasicAuthConfigurator.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/web/TestBasicAuthConfigurator.java index 16ed04ef4dacb..902c5262e1f72 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/web/TestBasicAuthConfigurator.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/web/TestBasicAuthConfigurator.java @@ -23,6 +23,8 @@ import java.io.IOException; import java.net.HttpURLConnection; +import java.nio.charset.StandardCharsets; +import java.util.Base64; public class TestBasicAuthConfigurator { @Test @@ -43,25 +45,39 @@ public void testEmptyCredentials() throws IOException { @Test public void testCredentialsSet() throws IOException { - ConnectionConfigurator conf = new BasicAuthConfigurator(null, "user:pass"); + String credentials = "user:pass"; + ConnectionConfigurator conf = new BasicAuthConfigurator(null, credentials); HttpURLConnection conn = Mockito.mock(HttpURLConnection.class); conf.configure(conn); Mockito.verify(conn, Mockito.times(1)).setRequestProperty( "AUTHORIZATION", - "Basic dXNlcjpwYXNz" + "Basic " + Base64.getEncoder().encodeToString(credentials.getBytes(StandardCharsets.UTF_8)) ); } @Test public void testParentConfigurator() throws IOException { ConnectionConfigurator parent = Mockito.mock(ConnectionConfigurator.class); - ConnectionConfigurator conf = new BasicAuthConfigurator(parent, "user:pass"); + String credentials = "user:pass"; + ConnectionConfigurator conf = new BasicAuthConfigurator(parent, credentials); HttpURLConnection conn = Mockito.mock(HttpURLConnection.class); conf.configure(conn); Mockito.verify(conn, Mockito.times(1)).setRequestProperty( "AUTHORIZATION", - "Basic dXNlcjpwYXNz" + "Basic " + Base64.getEncoder().encodeToString(credentials.getBytes(StandardCharsets.UTF_8)) ); Mockito.verify(parent, Mockito.times(1)).configure(conn); } + + @Test + public void testCredentialsSetWithUtfAndSpecialCharacters() throws IOException { + String credentials = "\uD80C\uDD04@ą:pass"; + ConnectionConfigurator conf = new BasicAuthConfigurator(null, credentials); + HttpURLConnection conn = Mockito.mock(HttpURLConnection.class); + conf.configure(conn); + Mockito.verify(conn, Mockito.times(1)).setRequestProperty( + "AUTHORIZATION", + "Basic " + Base64.getEncoder().encodeToString(credentials.getBytes(StandardCharsets.UTF_8)) + ); + } } From 9852e9ab7f2e0a78e2aebc353c6ab58354401832 Mon Sep 17 00:00:00 2001 From: Piotr Stankowski Date: Sun, 2 Apr 2023 00:13:50 +0200 Subject: [PATCH 6/7] Move BasicAuthConfigurator.getConfigurator to URLConnectionFactory.createBasicAuthConfigurator --- .../hdfs/web/BasicAuthConfigurator.java | 10 --------- .../hadoop/hdfs/web/URLConnectionFactory.java | 22 +++++++++++++++++-- 2 files changed, 20 insertions(+), 12 deletions(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/BasicAuthConfigurator.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/BasicAuthConfigurator.java index 67dcd2b861120..cb4879ca0a783 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/BasicAuthConfigurator.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/BasicAuthConfigurator.java @@ -33,16 +33,6 @@ public class BasicAuthConfigurator implements ConnectionConfigurator { private final ConnectionConfigurator parent; private final String credentials; - static public ConnectionConfigurator getConfigurator( - ConnectionConfigurator configurator, String basicAuthCred - ) { - if (basicAuthCred != null && !basicAuthCred.isEmpty()) { - return new BasicAuthConfigurator(configurator, basicAuthCred); - } else { - return configurator; - } - } - /** * @param credentials a string of the form "username:password" */ diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java index 9913b5a0d0018..3a60fff67f31a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java @@ -136,7 +136,7 @@ public HttpURLConnection configure(HttpURLConnection connection) } } - return BasicAuthConfigurator.getConfigurator(conn, basicAuthCredentials); + return createBasicAuthConfigurator(conn, basicAuthCredentials); } /** @@ -166,7 +166,7 @@ public static URLConnectionFactory newOAuth2URLConnectionFactory( } catch (Exception e) { throw new IOException("Unable to load OAuth2 connection factory.", e); } - conn = BasicAuthConfigurator.getConfigurator(conn, basicAuthCredentials); + conn = createBasicAuthConfigurator(conn, basicAuthCredentials); return new URLConnectionFactory(conn); } @@ -238,6 +238,24 @@ private static void setTimeouts(URLConnection connection, connection.setReadTimeout(readTimeout); } + /** + * Create a new ConnectionConfigurator wrapping the given one with HTTP Basic Authentication. + * It will return the original configurator if no credentials are specified. + * + * @param configurator Parent connection configurator. + * @param basicAuthCred Credentials for HTTP Basic Authentication in "username:password" format, + * or null / empty string if no credentials are to be used. + */ + private static ConnectionConfigurator createBasicAuthConfigurator( + ConnectionConfigurator configurator, String basicAuthCred + ) { + if (basicAuthCred != null && !basicAuthCred.isEmpty()) { + return new BasicAuthConfigurator(configurator, basicAuthCred); + } else { + return configurator; + } + } + public void destroy() { if (connConfigurator instanceof SSLConnectionConfigurator) { ((SSLConnectionConfigurator) connConfigurator).destroy(); From 57e1e566e51b25404c55c63ce74b4e7388075418 Mon Sep 17 00:00:00 2001 From: Piotr Stankowski Date: Sun, 2 Apr 2023 00:14:21 +0200 Subject: [PATCH 7/7] Make sure that uri isn't null --- .../main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java index b71bc84e0bca1..071c1a3e7d804 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java @@ -315,7 +315,7 @@ public StorageStatistics provide() { HdfsClientConfigKeys.DFS_CLIENT_WEBHDFS_USE_BASE_PATH_KEY, HdfsClientConfigKeys.DFS_CLIENT_WEBHDFS_USE_BASE_PATH_DEFAULT ); - if (uri.getPath() != null && !uri.getPath().equals("") && useBasePath) { + if (uri != null && uri.getPath() != null && !uri.getPath().equals("") && useBasePath) { pathPrefix = uri.getPath(); if (pathPrefix.endsWith("/")) { pathPrefix = pathPrefix.substring(0, pathPrefix.length() - 1);