-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-28903][STREAMING][PYSPARK][TESTS] Fix AWS JDK version conflict that breaks Pyspark Kinesis tests #25559
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
|
Test build #109592 has finished for PR 25559 at commit
|
|
Test build #109633 has finished for PR 25559 at commit
|
|
I'm pretty stumped on this. The NoSuchMethodError refers to a method that is definitely there. The AWS dependencies are harmonized. The Scala tests work. The dependency graph from SBT looks right. I am guessing something about how python tests work or are run is the issue, but not sure what would cause this, as it suggests a runtime AWS SDK version difference, but I don't see any other versions pulled in. |
|
Test build #109745 has finished for PR 25559 at commit
|
|
Test build #109748 has finished for PR 25559 at commit
|
|
Test build #109751 has finished for PR 25559 at commit
|
|
Well, I made a little breakthrough: if you build the Spark assembly, something puts |
|
That's probably the hadoop-cloud module if you have that profile enabled. |
|
Bingo, that's it. Thanks! I'll work on harmonizing its dependencies with the Kinesis module. |
| <groupId>com.fasterxml.jackson.core</groupId> | ||
| <artifactId>jackson-annotations</artifactId> | ||
| </exclusion> | ||
| <exclusion> |
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.
Preemptively CCing @steveloughran for a look at this. The TL;DR is that hadoop-cloud is brining in an old aws-java-sdk dependency to the assembly and it interferes with the Kinesis dependencies, which are newer. Excluding these is a bit extreme, but, the aws-java-sdk dependency brings in like 20 other AWS JARs. I'm not clear whether that's the intent anyway.
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.
Won't you break the hadoop-cloud profile by doing this?
The kinesis integration is not packaged as part of the Spark distribution (when you enable its profile), while hadoop-cloud is.
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.
Yeah, this is the thing. Right now we only pull in the core aws-java-sdk JAR. If I include aws-java-sdk as an explicit dependency, it pulls in tons of other dependencies that seem irrelevant to Spark. Hm, maybe I need to use <dependencyManagement> to more narrowly manage up the version of aws-java-sdk without affecting the transitive dependency resolution? Well, if this change works, at least we are on to the cause, and then I'll try that.
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.
1.7.4 is a really old version; hadoop 2.9+ uses a (fat) shaded JAR which has a consistent kinesis SDK in with it; 2.8 is on a 1.10.x I think
Go on, move off Hadoop 2.7 as a baseline. It's many years old. EOL/unsupported and never actually qualified against Java 8
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.
Thanks @steveloughran -- so, given that we are for better or worse here still on Hadoop 2.7 (because I think I need to back port this to 2.4 at least), is it safe to exclude the whole aws-java-sdk dependency? doesn't seem so as it would mean the user has to re-include it. But is it safe to assume they would be running this on Hadoop anyway?
Sounds like you are saying that in Hadoop 2.9, this dependency wouldn't exist or could be excluded.
So, excluding it definitely worked to solve the problem. Right now I'm seeing what happens if we explicitly manage its version up as a direct dependency because just managing it up with <dependencyManagement> wasn't enough. The downside is probably that the assembly brings in everything the aws-java-sdk depends on, which is a lot of stuff. We don't distribute the assembly per se (right?) so it doesn't really mean more careful checks of the license of all the dependencies.
Still, if somehow it were fine to exclude this dependency, that's the tidiest from Spark's perspective. Does that fly for Hadoop 2.7 or pretty well break the point of hadoop-cloud?
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.
+1 to excluding the AWS dependency. It is not actually something you can bundle into ASF releases anyway https://issues.apache.org/jira/browse/HADOOP-13794. But: it'd be good for a spark-hadoop-cloud artifact to be published with that dependency for downstream users, or at least the things you have to add documented somewhere.
FWIW, I do build and test the spark kinesis module as part of my AWS SDK update process -one that actually went pretty smoothly for a change last time. No regressions, no new error messages in logs, shaded JARs really are shaded, etc. This is progress and means that backporting is something we should be doing
see https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md#-qualifying-an-aws-sdk-update for the runbook there
|
Test build #109756 has finished for PR 25559 at commit
|
|
Test build #4841 has finished for PR 25559 at commit
|
|
Test build #4843 has finished for PR 25559 at commit
|
|
Test build #4844 has finished for PR 25559 at commit
|
|
That's good that this passes. Now going to try the less drastic change at #25559 (comment) to see if that works. |
|
Test build #109826 has finished for PR 25559 at commit
|
|
Test build #109832 has finished for PR 25559 at commit
|
|
Test build #4845 has finished for PR 25559 at commit
|
|
Test build #4846 has finished for PR 25559 at commit
|
|
Test build #109867 has finished for PR 25559 at commit
|
srowen
left a comment
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.
@steveloughran I'll re-run tests without my placeholder TODOs, but it won't matter (Kinesis tests won't run anyway). Looks like it passes with the exclusion. That's OK by you?
|
could some stick up the output of the mvn dependency listing here |
|
Base:
|
|
Test build #109879 has finished for PR 25559 at commit
|
|
@steveloughran does that seem reasonable? the above is after removing |
|
Per #25559 (comment) I'm going to proceed, to make the tests work again. If we need to make a more nuanced change we can do so in a follow-up. |
… that breaks Pyspark Kinesis tests
The Pyspark Kinesis tests are failing, at least in master:
```
======================================================================
ERROR: test_kinesis_stream (pyspark.streaming.tests.test_kinesis.KinesisStreamTests)
----------------------------------------------------------------------
Traceback (most recent call last):
File "/home/jenkins/workspace/SparkPullRequestBuilder2/python/pyspark/streaming/tests/test_kinesis.py", line 44, in test_kinesis_stream
kinesisTestUtils = self.ssc._jvm.org.apache.spark.streaming.kinesis.KinesisTestUtils(2)
File "/home/jenkins/workspace/SparkPullRequestBuilder2/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1554, in __call__
answer, self._gateway_client, None, self._fqn)
File "/home/jenkins/workspace/SparkPullRequestBuilder2/python/lib/py4j-0.10.8.1-src.zip/py4j/protocol.py", line 328, in get_return_value
format(target_id, ".", name), value)
Py4JJavaError: An error occurred while calling None.org.apache.spark.streaming.kinesis.KinesisTestUtils.
: java.lang.NoSuchMethodError: com.amazonaws.regions.Region.getAvailableEndpoints()Ljava/util/Collection;
at org.apache.spark.streaming.kinesis.KinesisTestUtils$.$anonfun$getRegionNameByEndpoint$1(KinesisTestUtils.scala:211)
at org.apache.spark.streaming.kinesis.KinesisTestUtils$.$anonfun$getRegionNameByEndpoint$1$adapted(KinesisTestUtils.scala:211)
at scala.collection.Iterator.find(Iterator.scala:993)
at scala.collection.Iterator.find$(Iterator.scala:990)
at scala.collection.AbstractIterator.find(Iterator.scala:1429)
at scala.collection.IterableLike.find(IterableLike.scala:81)
at scala.collection.IterableLike.find$(IterableLike.scala:80)
at scala.collection.AbstractIterable.find(Iterable.scala:56)
at org.apache.spark.streaming.kinesis.KinesisTestUtils$.getRegionNameByEndpoint(KinesisTestUtils.scala:211)
at org.apache.spark.streaming.kinesis.KinesisTestUtils.<init>(KinesisTestUtils.scala:46)
...
```
The non-Python Kinesis tests are fine though. It turns out that this is because Pyspark tests use the output of the Spark assembly, and it pulls in `hadoop-cloud`, which in turn pulls in an old AWS Java SDK.
Per Steve Loughran (below), it seems like we can just resolve this by excluding the aws-java-sdk dependency. See the attached PR for some more detail about the debugging and other options.
See #25558 (comment)
Closes #25559 from srowen/KinesisTest.
Authored-by: Sean Owen <[email protected]>
Signed-off-by: Sean Owen <[email protected]>
(cherry picked from commit d5b7eed)
Signed-off-by: Sean Owen <[email protected]>
|
Merged to master/2.4. If this turns out to cause problems for |
|
Thank you for this fix, @srowen ! |
|
sorry, been in hiding. hadoop branch-2 and cloud stuff is trouble as the aws sdk support has had to chase a moving target dependency-wise.
FWIW we use the apache hadoop-cloud-storage POM as the source of truth for which cloud store bits dependent apps pick up (spark, hive, etc); the spark hadoop-cloud POM pulls that in and then gets it into the spark releases. This provides a one-stop pipeline to get in things which aren't normally in releases (Google GCS) and leave out bits which aren't currently supports (allyun OSS). That generally keeps things under control, leaving only configuration settings... |
… that breaks Pyspark Kinesis tests
The Pyspark Kinesis tests are failing, at least in master:
```
======================================================================
ERROR: test_kinesis_stream (pyspark.streaming.tests.test_kinesis.KinesisStreamTests)
----------------------------------------------------------------------
Traceback (most recent call last):
File "/home/jenkins/workspace/SparkPullRequestBuilder2/python/pyspark/streaming/tests/test_kinesis.py", line 44, in test_kinesis_stream
kinesisTestUtils = self.ssc._jvm.org.apache.spark.streaming.kinesis.KinesisTestUtils(2)
File "/home/jenkins/workspace/SparkPullRequestBuilder2/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1554, in __call__
answer, self._gateway_client, None, self._fqn)
File "/home/jenkins/workspace/SparkPullRequestBuilder2/python/lib/py4j-0.10.8.1-src.zip/py4j/protocol.py", line 328, in get_return_value
format(target_id, ".", name), value)
Py4JJavaError: An error occurred while calling None.org.apache.spark.streaming.kinesis.KinesisTestUtils.
: java.lang.NoSuchMethodError: com.amazonaws.regions.Region.getAvailableEndpoints()Ljava/util/Collection;
at org.apache.spark.streaming.kinesis.KinesisTestUtils$.$anonfun$getRegionNameByEndpoint$1(KinesisTestUtils.scala:211)
at org.apache.spark.streaming.kinesis.KinesisTestUtils$.$anonfun$getRegionNameByEndpoint$1$adapted(KinesisTestUtils.scala:211)
at scala.collection.Iterator.find(Iterator.scala:993)
at scala.collection.Iterator.find$(Iterator.scala:990)
at scala.collection.AbstractIterator.find(Iterator.scala:1429)
at scala.collection.IterableLike.find(IterableLike.scala:81)
at scala.collection.IterableLike.find$(IterableLike.scala:80)
at scala.collection.AbstractIterable.find(Iterable.scala:56)
at org.apache.spark.streaming.kinesis.KinesisTestUtils$.getRegionNameByEndpoint(KinesisTestUtils.scala:211)
at org.apache.spark.streaming.kinesis.KinesisTestUtils.<init>(KinesisTestUtils.scala:46)
...
```
The non-Python Kinesis tests are fine though. It turns out that this is because Pyspark tests use the output of the Spark assembly, and it pulls in `hadoop-cloud`, which in turn pulls in an old AWS Java SDK.
Per Steve Loughran (below), it seems like we can just resolve this by excluding the aws-java-sdk dependency. See the attached PR for some more detail about the debugging and other options.
See apache#25558 (comment)
Closes apache#25559 from srowen/KinesisTest.
Authored-by: Sean Owen <[email protected]>
Signed-off-by: Sean Owen <[email protected]>
(cherry picked from commit d5b7eed)
Signed-off-by: Sean Owen <[email protected]>
The Pyspark Kinesis tests are failing, at least in master:
The non-Python Kinesis tests are fine though. It turns out that this is because Pyspark tests use the output of the Spark assembly, and it pulls in
hadoop-cloud, which in turn pulls in an old AWS Java SDK.Per Steve Loughran (below), it seems like we can just resolve this by excluding the aws-java-sdk dependency. See the attached PR for some more detail about the debugging and other options.
See #25558 (comment)