[ 
https://issues.apache.org/jira/browse/BEAM-11626?focusedWorklogId=540379&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-540379
 ]

ASF GitHub Bot logged work on BEAM-11626:
-----------------------------------------

                Author: ASF GitHub Bot
            Created on: 22/Jan/21 22:40
            Start Date: 22/Jan/21 22:40
    Worklog Time Spent: 10m 
      Work Description: suztomo commented on a change in pull request #13740:
URL: https://github.com/apache/beam/pull/13740#discussion_r562954781



##########
File path: CHANGES.md
##########
@@ -67,6 +67,11 @@
 
 ## Breaking Changes
 
+* The Java artifacts "beam-sdks-java-io-kinesis", 
"beam-sdks-java-io-google-cloud-platform", and
+  "beam-sdks-java-extensions-sql-zetasql" declare Guava 30.1-jre dependency 
(It was 25.1-jre in Beam 2.27.0).
+  This new Guava version may introduce dependency conflicts if your project or 
dependencies rely
+  on removed APIs. If affected, ensure to use an appropriate Guava version via 
`dependencyManagement` in Maven and
+  `force` in Gradle.

Review comment:
       @aaltay I added this note for potential impact to Beam users. The 
potential risk described here is not special to this Guava version. Every 
dependency upgrade, in general, carries a risk of introducing dependency 
conflicts if a user relies on removed methods or classes. (Therefore this note 
might not be needed.)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
[email protected]


Issue Time Tracking
-------------------

    Worklog Id:     (was: 540379)
    Time Spent: 14h  (was: 13h 50m)

> Upgrade Guava to the latest version except for Hadoop and Cassandra
> -------------------------------------------------------------------
>
>                 Key: BEAM-11626
>                 URL: https://issues.apache.org/jira/browse/BEAM-11626
>             Project: Beam
>          Issue Type: Improvement
>          Components: build-system
>            Reporter: Tomo Suzuki
>            Assignee: Tomo Suzuki
>            Priority: P2
>          Time Spent: 14h
>  Remaining Estimate: 0h
>
> Related to https://issues.apache.org/jira/browse/BEAM-5559
> As per BEAM-8911, Cassandra-all requires Guava 25.1. But other parts of Beam 
> dependencies (such as gcsio) may want to use newer version of Guava. Excerpt 
> from Kiley's finding using Linkage Checker today:
> {noformat}
> (com.google.guava:guava:25.1-jre) 
> com.google.common.util.concurrent.Uninterruptibles's method 
> sleepUninterruptibly(java.time.Duration) is not found;
>   referenced by 3 class files
>     com.google.cloud.hadoop.gcsio.cooplock.CoopLockOperationDao 
> (com.google.cloud.bigdataoss:gcsio:2.1.6)
>     com.google.cloud.hadoop.gcsio.cooplock.CoopLockRecordsDao 
> (com.google.cloud.bigdataoss:gcsio:2.1.6)
>     com.google.cloud.hadoop.gcsio.testing.InMemoryObjectEntry 
> (com.google.cloud.bigdataoss:gcsio:2.1.6)
> {noformat}
> Note that Guava's 
> [Uninterruptibles.sleepUninterruptibly(Duration)|https://guava.dev/releases/30.1-jre/api/docs/com/google/common/util/concurrent/Uninterruptibles.html#sleepUninterruptibly(java.time.Duration)]
>  has been introduced since 28.0.
> This ticket is attempts to declare the newer version of Guava while keeping 
> the Guava version used by Beam's Cassandra tests.
> h1. Where is Guava dependency declared?
> {noformat}
> suztomo-macbookpro44% find . -name 'build.gradle' |xargs grep 
> 'library.java.guava'
> ./sdks/java/core/build.gradle:  shadowTest library.java.guava_testlib
> ./sdks/java/io/kinesis/build.gradle:  compile library.java.guava
> ./sdks/java/io/kinesis/build.gradle:  testCompile library.java.guava_testlib
> ./sdks/java/io/amazon-web-services2/build.gradle:  testCompile 
> library.java.guava_testlib
> ./sdks/java/io/google-cloud-platform/build.gradle:  compile library.java.guava
> ./sdks/java/io/contextualtextio/build.gradle:    testCompile 
> library.java.guava_testlib
> ./sdks/java/extensions/sql/zetasql/build.gradle:  compile library.java.guava
> ./sdks/java/maven-archetypes/examples/build.gradle:    'guava.version': 
> dependencies.create(project.library.java.guava).getVersion(),
> ./runners/google-cloud-dataflow-java/build.gradle:  testCompile 
> library.java.guava_testlib
> {noformat}
> Other than tests, {{sdks/java/io/kinesis}},  
> {{sdks/java/io/google-cloud-platform}}, {{sdks/java/extensions/sql/zetasql}} 
> will start declaring the new Guava dependency.
> The {{sdks/java/maven-archetypes/examples}} is tricky one. I want 
> Hadoop/Cassandra users to use Guava 25.1 and others to use Guava 30.
> h1. Failures with Guava 30.1-jre
> With simply upgrading Guava version to 30.1-jre 
> ([35f5b0|https://github.com/apache/beam/commit/35f5b0597e62563d843c9b3a7876212967b0a63c]),
>  I get the following errors:
> https://gist.github.com/suztomo/f7bbf77b54265eb80fd3c2d230b678dc
> Failures in my local tests:
> org.apache.beam.sdk.io.hdfs.HadoopFileSystemRegistrarTest and other tests in 
> the package:
> {noformat}
> java.lang.NoSuchMethodError: 
> com.google.common.util.concurrent.Futures.addCallback(Lcom/google/common/util/concurrent/ListenableFuture;Lcom/google/common/util/concurrent/FutureCallback;)V
>       at 
> org.apache.hadoop.hdfs.server.datanode.checker.ThrottledAsyncChecker.addResultCachingCallback(ThrottledAsyncChecker.java:167)
>       at 
> org.apache.hadoop.hdfs.server.datanode.checker.ThrottledAsyncChecker.schedule(ThrottledAsyncChecker.java:156)
>       at 
> org.apache.hadoop.hdfs.server.datanode.checker.StorageLocationChecker.check(StorageLocationChecker.java:165)
>       at 
> org.apache.hadoop.hdfs.server.datanode.DataNode.makeInstance(DataNode.java:2703)
>       at 
> org.apache.hadoop.hdfs.server.datanode.DataNode.instantiateDataNode(DataNode.java:2613)
>       at 
> org.apache.hadoop.hdfs.MiniDFSCluster.startDataNodes(MiniDFSCluster.java:1683)
>       at 
> org.apache.hadoop.hdfs.MiniDFSCluster.initMiniDFSCluster(MiniDFSCluster.java:905)
>       at org.apache.hadoop.hdfs.MiniDFSCluster.<init>(MiniDFSCluster.java:517)
>       at 
> org.apache.hadoop.hdfs.MiniDFSCluster$Builder.build(MiniDFSCluster.java:476)
>       at 
> org.apache.beam.sdk.io.hdfs.HadoopFileSystemRegistrarTest.setUp(HadoopFileSystemRegistrarTest.java:58)
>       at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> {noformat}
> This error appears in the following tests in sdks/java/io/hadoop-file-system 
> module:
> {noformat}
> Test Result (22 failures / +22)
> org.apache.beam.sdk.io.hdfs.HadoopFileSystemRegistrarTest.testServiceLoader
> org.apache.beam.sdk.io.hdfs.HadoopFileSystemTest.testDeleteNonExisting
> org.apache.beam.sdk.io.hdfs.HadoopFileSystemTest.testRenameExistingDestination
> org.apache.beam.sdk.io.hdfs.HadoopFileSystemTest.testMatch
> org.apache.beam.sdk.io.hdfs.HadoopFileSystemTest.testCopy
> org.apache.beam.sdk.io.hdfs.HadoopFileSystemTest.testMatchForNonExistentFile
> org.apache.beam.sdk.io.hdfs.HadoopFileSystemTest.testCreateAndReadFile
> org.apache.beam.sdk.io.hdfs.HadoopFileSystemTest.testMatchDirectory
> org.apache.beam.sdk.io.hdfs.HadoopFileSystemTest.testRenameRetryScenario
> org.apache.beam.sdk.io.hdfs.HadoopFileSystemTest.testRenameMissingTargetDir
> org.apache.beam.sdk.io.hdfs.HadoopFileSystemTest.testCreateAndReadFileWithShift
> org.apache.beam.sdk.io.hdfs.HadoopFileSystemTest.testCreateAndReadFileWithShiftToEnd
> org.apache.beam.sdk.io.hdfs.HadoopFileSystemTest.testCopySourceMissing
> org.apache.beam.sdk.io.hdfs.HadoopFileSystemTest.testRenameMissingSource
> org.apache.beam.sdk.io.hdfs.HadoopFileSystemTest.testMatchNewResource
> org.apache.beam.sdk.io.hdfs.HadoopFileSystemTest.testMatchForRecursiveGlob
> org.apache.beam.sdk.io.hdfs.HadoopFileSystemTest.testDelete
> org.apache.beam.sdk.io.hdfs.HadoopFileSystemTest.testRename
> org.apache.beam.sdk.io.hdfs.HadoopResourceIdTest.testGetFilename
> org.apache.beam.sdk.io.hdfs.HadoopResourceIdTest.testResourceIdTester
> {noformat}
> The deprecated 
> [Futures.addCallback|https://guava.dev/releases/25.1-jre/api/docs/com/google/common/util/concurrent/Futures.html]
>  has been removed since 26.0.
> and another error due to Guava version:
> {noformat}
> java.lang.RuntimeException: java.util.concurrent.ExecutionException: 
> java.lang.NoSuchFieldError: DIGIT
>       at 
> org.apache.cassandra.utils.FBUtilities.waitOnFuture(FBUtilities.java:442)
>       at 
> org.apache.cassandra.db.SystemKeyspace.forceBlockingFlush(SystemKeyspace.java:829)
>       at 
> org.apache.cassandra.db.SystemKeyspace.removeTruncationRecord(SystemKeyspace.java:668)
>       at 
> org.apache.cassandra.db.ColumnFamilyStore.invalidate(ColumnFamilyStore.java:568)
>       at 
> org.apache.cassandra.db.ColumnFamilyStore.invalidate(ColumnFamilyStore.java:544)
>       at 
> org.apache.cassandra.schema.LegacySchemaMigrator.lambda$unloadLegacySchemaTables$1(LegacySchemaMigrator.java:137)
>       at 
> com.google.common.collect.ImmutableList.forEach(ImmutableList.java:406)
>       at 
> org.apache.cassandra.schema.LegacySchemaMigrator.unloadLegacySchemaTables(LegacySchemaMigrator.java:137)
>       at 
> org.apache.cassandra.schema.LegacySchemaMigrator.migrate(LegacySchemaMigrator.java:83)
>       at 
> org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:252)
>       at 
> org.apache.cassandra.service.CassandraDaemon.init(CassandraDaemon.java:513)
>       at 
> org.apache.cassandra.service.EmbeddedCassandraService.start(EmbeddedCassandraService.java:51)
>       at 
> org.apache.beam.sdk.io.hadoop.format.HadoopFormatIOCassandraTest.beforeClass(HadoopFormatIOCassandraTest.java:192)
>       at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> {noformat}
> This error appear in this test in sdks/java/io/hadoop-format module:
> {noformat}
> org.apache.beam.sdk.io.hadoop.format.HadoopFormatIOCassandraTest.classMethod
> {noformat}
> This should be reproducible by {{./gradlew -p sdks/java/io/hadoop-format 
> cleanTest test}}
> The {{sdks/java/io/cassandra}} module also had the problem:
> {noformat}
> ./gradlew sdks:java:io:cassandra:test --debug --stacktrace
> ...
> 2021-01-14T00:30:29.077-0500 [DEBUG] [TestEventLogger]     Caused by: 
> java.lang.NoSuchFieldError: DIGIT
> 2021-01-14T00:30:29.077-0500 [DEBUG] [TestEventLogger]          at 
> org.apache.cassandra.io.sstable.format.SSTableFormat$Type.<init>(SSTableFormat.java:63)
> 2021-01-14T00:30:29.077-0500 [DEBUG] [TestEventLogger]          at 
> org.apache.cassandra.io.sstable.format.SSTableFormat$Type.<clinit>(SSTableFormat.java:46)
> 2021-01-14T00:30:29.086-0500 [DEBUG] [TestEventLogger]          at 
> org.apache.cassandra.db.ColumnFamilyStore.getSSTablePath(ColumnFamilyStore.java:843)
> 2021-01-14T00:30:29.087-0500 [DEBUG] [TestEventLogger]          at 
> org.apache.cassandra.db.Memtable$FlushRunnable.<init>(Memtable.java:443)
> {noformat}
> Dependency graph of the hadoop-format module: 
> https://gist.github.com/suztomo/8a444075d50e21cffab7bc9f087c82f3.
> h1. Probably Irrelevant Cassandra Build Failure in my Linux host
> The build (master) fails in my Linux machine for Cassandra test:
> {noformat}
> suztomo@suztomo:~/beam$ ./gradlew :sdks:java:io:cassandra:test
> Configuration on demand is an incubating feature.
> > Task :sdks:java:io:cassandra:test
> #
> # A fatal error has been detected by the Java Runtime Environment:
> #
> #  SIGSEGV (0xb) at pc=0x00007fbdbc875a3f, pid=3453267, tid=0x000000000034b177
> #
> # JRE version: OpenJDK Runtime Environment (8.0_181) (build 
> 1.8.0_181-google-v7-341329124-341329124)
> # Java VM: OpenJDK 64-Bit Server VM (25.181-b01 mixed mode linux-amd64 
> compressed oops)
> # Problematic frame:
> # C  [libc.so.6+0xbaa3f]  __gettimeofday+0x7f
> #
> # Failed to write core dump. Core dumps have been disabled. To enable core 
> dumping, try "ulimit -c unlimited" before starting Java again
> ...
> {noformat}
> {{:sdks:java:io:cassandra:test}} works in my MacBook Pro. Other tests in 
> {{:sdks:java:io}} passed in Linux.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to