-
Notifications
You must be signed in to change notification settings - Fork 28.6k
[SPARK-30272][SQL][CORE] Remove usage of Guava that breaks in 27; replace with workalikes #26911
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
Conversation
common/kvstore/src/test/java/org/apache/spark/util/kvstore/CustomType1.java
Show resolved
Hide resolved
.add("file", file) | ||
.add("offset", offset) | ||
.add("length", length) | ||
return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The format of the toString is a bit different after this. Before (Guava): Foo{bar=baz, bing=3}
After (Commons Lang3): Foo[bar=baz,bing=3]
. I think the exact format is not important for these classes though?
@@ -54,7 +57,7 @@ public static ChunkFetchFailure decode(ByteBuf buf) { | |||
|
|||
@Override | |||
public int hashCode() { | |||
return Objects.hashCode(streamChunkId, errorString); | |||
return Objects.hash(streamChunkId, errorString); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Work-alikes from java.util; should do the same thing as it calls Arrays.hashCode in both cases
@@ -42,7 +41,7 @@ | |||
public class CleanupNonShuffleServiceServedFilesSuite { | |||
|
|||
// Same-thread Executor used to ensure cleanup happens synchronously in test thread. | |||
private Executor sameThreadExecutor = MoreExecutors.sameThreadExecutor(); | |||
private Executor sameThreadExecutor = Runnable::run; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This makes an Executor that just runs the Runnable
@@ -76,7 +76,8 @@ class SparkEnv ( | |||
|
|||
// A general, soft-reference map for metadata needed during HadoopRDD split computation | |||
// (e.g., HadoopFileRDD uses this to cache JobConfs and InputFormats). | |||
private[spark] val hadoopJobMetadata = new MapMaker().softValues().makeMap[String, Any]() | |||
private[spark] val hadoopJobMetadata = |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This object was really a cache anyway and it was recommended to use CacheBuilder instead of MapMaker.
@@ -405,9 +405,9 @@ private[spark] object HadoopRDD extends Logging { | |||
* The three methods below are helpers for accessing the local map, a property of the SparkEnv of | |||
* the local process. | |||
*/ | |||
def getCachedMetadata(key: String): Any = SparkEnv.get.hadoopJobMetadata.get(key) | |||
def getCachedMetadata(key: String): AnyRef = SparkEnv.get.hadoopJobMetadata.get(key) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Problem here was using a non-ref type in a Java Map. This is only really used for a JobConf anyway
Thank you so much, @srowen ! |
Test build #115404 has finished for PR 26911 at commit
|
Test build #115405 has finished for PR 26911 at commit
|
Test build #115412 has finished for PR 26911 at commit
|
Test build #115406 has finished for PR 26911 at commit
|
Test build #115462 has finished for PR 26911 at commit
|
Test build #115476 has finished for PR 26911 at commit
|
Merged to master |
…gBuilder ### What changes were proposed in this pull request? This PR replaces `toStringHelper`, an API which breaks in Guava 27. ### Why are the changes needed? SPARK-30272 (#26911) removed usages which breaks in Guava 27 but `toStringHelper` is instroduced again. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Build successfully finished with the following command. ``` build/sbt -Dguava.version=27.0-jre -Phive -Phive-thriftserver -Pyarn -Pmesos -Pkubernetes -Phadoop-cloud -Pdocker-integration-tests -Pkubernetes-integration-tests -Pkinesis-asl -Pspark-ganglia-lgpl package ``` Closes #32567 from sarutak/remove-old-guava-usage. Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com> Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
… which is incompatible with newer versions ### What changes were proposed in this pull request? This PR adds rules to `checkstyle.xml` and `scalastyle-config.xml` to avoid introducing `Objects.toStringHelper` a Guava's API which is no longer present in newer Guava. ### Why are the changes needed? SPARK-30272 (#26911) replaced `Objects.toStringHelper` which is an APIs Guava 14 provides with `commons.lang3` API because `Objects.toStringHelper` is no longer present in newer Guava. But toStringHelper was introduced into Spark again and replaced them in SPARK-35420 (#32567). I think it's better to have a style rule to avoid such repetition. SPARK-30272 replaced some APIs aside from `Objects.toStringHelper` but `Objects.toStringHelper` seems to affect Spark for now so I add rules only for it. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? I confirmed that `lint-java` and `lint-scala` detect the usage of `toStringHelper` and let the lint check fail. ``` $ dev/lint-java exec: curl --silent --show-error -L https://downloads.lightbend.com/scala/2.12.14/scala-2.12.14.tgz Using `mvn` from path: /opt/maven/3.6.3//bin/mvn Checkstyle checks failed at following occurrences: [ERROR] src/main/java/org/apache/spark/network/protocol/OneWayMessage.java:[78] (regexp) RegexpSinglelineJava: Avoid using Object.toStringHelper. Use ToStringBuilder instead. $ dev/lint-scala Scalastyle checks failed at following occurrences: [error] /home/kou/work/oss/spark/core/src/main/scala/org/apache/spark/rdd/RDD.scala:93:25: Avoid using Object.toStringHelper. Use ToStringBuilder instead. [error] Total time: 25 s, completed 2021/06/02 16:18:25 ``` Closes #32740 from sarutak/style-rule-for-guava. Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com> Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
### What changes were proposed in this pull request? As title ### Why are the changes needed? - bump guava from 14.0.1 to 32.1.3-jre - refer to apache/spark#26911, remove usages of Guava that no longer work in Guava 27/32, and replace with workalikes. After this PR, Celeborn no longer relies on a specific version of Guava, and is compatible with Guava 14/27/32. we have the ability to specify Guava to 27 when running MapReduce integration tests. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pass GA Closes #2090 from cfmcgrady/guava-27. Authored-by: Fu Chen <cfmcgrady@gmail.com> Signed-off-by: mingji <fengmingxiao.fmx@alibaba-inc.com>
What changes were proposed in this pull request?
Remove usages of Guava that no longer work in Guava 27, and replace with workalikes. I'll comment on key types of changes below.
Why are the changes needed?
Hadoop 3.2.1 uses Guava 27, so this helps us avoid problems running on Hadoop 3.2.1+ and generally lowers our exposure to Guava.
Does this PR introduce any user-facing change?
Should not be, but see notes below on hash codes and toString.
How was this patch tested?
Existing tests will verify whether these changes break anything for Guava 14.
I manually built with an updated version and it compiles with Guava 27; tests running manually locally now.