This is an automated email from the ASF dual-hosted git repository. dongjoon pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git
The following commit(s) were added to refs/heads/master by this push: new 6eb078099739 [SPARK-53237][SQL] Use Java `Base64` instead of `org.apache.commons.codec.binary.Base64` instance 6eb078099739 is described below commit 6eb078099739e4045184e65acfb79dd921c41421 Author: Dongjoon Hyun <dongj...@apache.org> AuthorDate: Sun Aug 10 19:27:30 2025 -0700 [SPARK-53237][SQL] Use Java `Base64` instead of `org.apache.commons.codec.binary.Base64` instance ### What changes were proposed in this pull request? This PR aims to use Java `Base64` instead of `org.apache.commons.codec.binary.Base64`. This is similar to the following one. - https://github.com/apache/spark/pull/51904 - https://github.com/apache/spark/pull/51938 In addition, the existing Scalastyle is revised and a new Checkstyle rules ar added to ban `org.apache.commons.codec.binary.Base64` in order to prevent a future regression. ### Why are the changes needed? Java native implementation is **roughly 3x faster** than `Apache Commons` one. ```scala scala> val s = "a".repeat(5_000_000).getBytes(java.nio.charset.StandardCharsets.UTF_8) scala> spark.time(java.util.Base64.getDecoder().decode(java.util.Base64.getEncoder().encodeToString(s)).length) Time taken: 21 ms val res0: Int = 5000000 scala> spark.time(new org.apache.commons.codec.binary.Base64(0).decode(new org.apache.commons.codec.binary.Base64(0).encodeToString(s)).length) Time taken: 60 ms val res1: Int = 5000000 ``` ### Does this PR introduce _any_ user-facing change? No behavior change. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #51963 from dongjoon-hyun/SPARK-53237. Authored-by: Dongjoon Hyun <dongj...@apache.org> Signed-off-by: Dongjoon Hyun <dongj...@apache.org> --- dev/checkstyle.xml | 1 + scalastyle-config.xml | 2 +- .../src/main/java/org/apache/hive/service/CookieSigner.java | 5 ++--- .../src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java | 6 ++---- 4 files changed, 6 insertions(+), 8 deletions(-) diff --git a/dev/checkstyle.xml b/dev/checkstyle.xml index 13b753c671db..7e47a395b1ac 100644 --- a/dev/checkstyle.xml +++ b/dev/checkstyle.xml @@ -184,6 +184,7 @@ <property name="illegalPkgs" value="org.apache.log4j" /> <property name="illegalPkgs" value="org.apache.commons.lang" /> <property name="illegalPkgs" value="org.apache.commons.lang3.tuple" /> + <property name="illegalClasses" value="org.apache.commons.codec.binary.Base64" /> <property name="illegalClasses" value="org.apache.commons.io.FileUtils" /> <property name="illegalClasses" value="org.apache.commons.lang3.JavaVersion" /> <property name="illegalClasses" value="org.apache.commons.lang3.Strings" /> diff --git a/scalastyle-config.xml b/scalastyle-config.xml index 0d0bc722ba23..4c0439b73c73 100644 --- a/scalastyle-config.xml +++ b/scalastyle-config.xml @@ -444,7 +444,7 @@ This file is divided into 3 sections: </check> <check customId="commonscodecbase64" level="error" class="org.scalastyle.file.RegexChecker" enabled="true"> - <parameters><parameter name="regex">\bBase64\.(en|de)codeBase64</parameter></parameters> + <parameters><parameter name="regex">org\.apache\.commons\.codec\.binary\.Base64\b</parameter></parameters> <customMessage>Use java.util.Base64 instead</customMessage> </check> diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/CookieSigner.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/CookieSigner.java index 0572ec5ac08f..7dbc7063e891 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/CookieSigner.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/CookieSigner.java @@ -19,8 +19,7 @@ package org.apache.hive.service; import java.security.MessageDigest; import java.security.NoSuchAlgorithmException; - -import org.apache.commons.codec.binary.Base64; +import java.util.Base64; import org.apache.spark.internal.SparkLogger; import org.apache.spark.internal.SparkLoggerFactory; @@ -95,7 +94,7 @@ public class CookieSigner { md.update(str.getBytes()); md.update(secretBytes); byte[] digest = md.digest(); - return new Base64(0).encodeToString(digest); + return Base64.getEncoder().encodeToString(digest); } catch (NoSuchAlgorithmException ex) { throw new RuntimeException("Invalid SHA digest String: " + SHA_STRING + " " + ex.getMessage(), ex); diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java index a76bcc9b873e..0a306abed509 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java @@ -22,6 +22,7 @@ import java.security.AccessController; import java.security.PrivilegedExceptionAction; import java.security.SecureRandom; import java.util.Arrays; +import java.util.Base64; import java.util.HashMap; import java.util.HashSet; import java.util.Map; @@ -30,7 +31,6 @@ import java.util.StringTokenizer; import javax.security.auth.Subject; -import org.apache.commons.codec.binary.Base64; import org.apache.hadoop.hive.shims.ShimLoader; import org.apache.hadoop.security.UserGroupInformation; import org.apache.http.protocol.BasicHttpContext; @@ -157,13 +157,11 @@ public final class HttpAuthUtils { public static final String SERVER_HTTP_URL = "SERVER_HTTP_URL"; private final String serverPrincipal; private final String serverHttpUrl; - private final Base64 base64codec; private final HttpContext httpContext; public HttpKerberosClientAction(String serverPrincipal, String serverHttpUrl) { this.serverPrincipal = serverPrincipal; this.serverHttpUrl = serverHttpUrl; - base64codec = new Base64(0); httpContext = new BasicHttpContext(); httpContext.setAttribute(SERVER_HTTP_URL, serverHttpUrl); } @@ -187,7 +185,7 @@ public final class HttpAuthUtils { byte[] outToken = gssContext.initSecContext(inToken, 0, inToken.length); gssContext.dispose(); // Base64 encoded and stringified token for server - return new String(base64codec.encode(outToken)); + return Base64.getEncoder().encodeToString(outToken); } } } --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org