The protobuf bug summarizes the issue well. It is built against Java 11
bootclasspath even though it targets bytecode compatibility with Java 8.

It seems like we intend to be at 3.12.0 by default:
https://github.com/apache/beam/blob/550ea9fc4db38f543350350fcc0734029a587e81/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy#L430

Kenn

On Fri, Nov 6, 2020 at 4:16 PM Steve Niemitz <sniem...@twitter.com> wrote:

> I downgraded google_cloud_bigdataoss from 2.1.5 back to 2.1.3, which was
> recently upgraded [1], and that fixed the issue.  It looks like it was
> transitively pulling in protobuf 3.13.0, which isn't compatible with java
> 8(?!??).
>
> [1]
> https://github.com/apache/beam/commit/7fec038bf9e3861462744ba5522208a4b9d15b85#diff-0435a83a413ec063bf7e682cadcd56776cd18fc878f197cc99a65fc231ef2047
>
> On Fri, Nov 6, 2020 at 6:27 PM Steve Niemitz <sniem...@apache.org> wrote:
>
>> yeah, I built it via:
>> JAVA_HOME=/usr/lib/jvm/java-1.8.0-openjdk-amd64 ./gradlew --no-daemon
>> -Ppublishing -PnoSigning publishMavenJavaPublicationToMavenLocal
>>
>> For me java8 is also my default
>>
>> On Fri, Nov 6, 2020 at 6:25 PM Kyle Weaver <kcwea...@google.com> wrote:
>>
>>> Do you have JAVA_HOME set? (possibly related:
>>> https://issues.apache.org/jira/browse/BEAM-11080)
>>>
>>> On Fri, Nov 6, 2020 at 3:13 PM Steve Niemitz <sniem...@apache.org>
>>> wrote:
>>>
>>>> I'm trying out 2.25 (built from source, using java 8), and running into
>>>> this error, both on the direct runner and dataflow:
>>>>
>>>> Caused by: java.lang.NoSuchMethodError:
>>>> java.nio.ByteBuffer.position(I)Ljava/nio/ByteBuffer;
>>>> at
>>>> com.google.protobuf.NioByteString.copyToInternal(NioByteString.java:112)
>>>> at com.google.protobuf.ByteString.toByteArray(ByteString.java:695)
>>>> at com.google.protobuf.NioByteString.writeTo(NioByteString.java:123)
>>>> at
>>>> org.apache.beam.sdk.extensions.protobuf.ByteStringCoder.encode(ByteStringCoder.java:67)
>>>> at
>>>> org.apache.beam.sdk.extensions.protobuf.ByteStringCoder.encode(ByteStringCoder.java:37)
>>>> at
>>>> org.apache.beam.sdk.coders.DelegateCoder.encode(DelegateCoder.java:74)
>>>> at
>>>> org.apache.beam.sdk.coders.DelegateCoder.encode(DelegateCoder.java:68)
>>>>
>>>> It seems like this was introduced in protobuf 3.12.4 based on this
>>>> issue I found [1]
>>>>
>>>> Am I doing something wrong with my build? Or am I just hitting an
>>>> untested combo here?
>>>>
>>>> [1] https://github.com/protocolbuffers/protobuf/issues/7827
>>>>
>>>

Reply via email to