You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2022/09/02 14:52:13 UTC

[GitHub] [beam] takaaki7 opened a new issue, #23009: [Bug]: When io.grpc:grpc-netty-shaded:1.47.0>= is included in dependency, using DataflowRunnerV2 and BigQueryIO, exception thrown.

takaaki7 opened a new issue, #23009:
URL: https://github.com/apache/beam/issues/23009

   ### What happened?
   
   When io.grpc:grpc-netty-shaded:1.47.0>= is included in dependency, using DataflowRunnerV2 and BigQueryIO, pipeline always fails.
   
   e.g. Following dependency includes io.grpc:grpc-netty-shaded:1.48.0.
   ```gradle
       implementation 'com.google.cloud:google-cloud-bigquery:2.14.0'
       implementation 'org.apache.beam:beam-sdks-java-core:2.39.0'
       implementation 'org.apache.beam:beam-runners-google-cloud-dataflow-java:2.39.0'
       implementation 'org.apache.beam:beam-runners-direct-java:2.39.0'
       implementation 'org.apache.beam:beam-sdks-java-io-google-cloud-platform:2.39.0'
       implementation 'org.apache.beam:beam-sdks-java-extensions-google-cloud-platform-core:2.39.0'
   ```
   
   And using BigQueryIO, pipeline fails with following error.
   ```
   022-09-02 23:26:55.961 JSTError message from worker:
   org.apache.beam.sdk.util.UserCodeException: java.lang.NoSuchFieldError: EPOLL_DOMAIN_CLIENT_CHANNEL_TYPE
   org.apache.beam.sdk.util.UserCodeException.wrap(UserCodeException.java:39)
   org.apache.beam.sdk.io.Read$BoundedSourceAsSDFWrapperFn$DoFnInvoker.invokeSplitRestriction(Unknown Source)
   org.apache.beam.fn.harness.FnApiDoFnRunner.processElementForSplitRestriction(FnApiDoFnRunner.java:861)
   org.apache.beam.fn.harness.data.PCollectionConsumerRegistry$MetricTrackingFnDataReceiver.accept(PCollectionConsumerRegistry.java:255)
   org.apache.beam.fn.harness.data.PCollectionConsumerRegistry$MetricTrackingFnDataReceiver.accept(PCollectionConsumerRegistry.java:209)
   org.apache.beam.fn.harness.FnApiDoFnRunner.outputTo(FnApiDoFnRunner.java:1760)
   org.apache.beam.fn.harness.FnApiDoFnRunner.processElementForPairWithRestriction(FnApiDoFnRunner.java:798)
   org.apache.beam.fn.harness.data.PCollectionConsumerRegistry$MetricTrackingFnDataReceiver.accept(PCollectionConsumerRegistry.java:255)
   org.apache.beam.fn.harness.data.PCollectionConsumerRegistry$MetricTrackingFnDataReceiver.accept(PCollectionConsumerRegistry.java:209)
   org.apache.beam.fn.harness.FnApiDoFnRunner.outputTo(FnApiDoFnRunner.java:1760)
   org.apache.beam.fn.harness.FnApiDoFnRunner.access$2800(FnApiDoFnRunner.java:142)
   org.apache.beam.fn.harness.FnApiDoFnRunner$NonWindowObservingProcessBundleContext.outputWithTimestamp(FnApiDoFnRunner.java:2311)
   org.apache.beam.fn.harness.FnApiDoFnRunner$ProcessBundleContextBase.output(FnApiDoFnRunner.java:2481)
   org.apache.beam.sdk.transforms.DoFnOutputReceivers$WindowedContextOutputReceiver.output(DoFnOutputReceivers.java:78)
   org.apache.beam.sdk.io.Read$OutputSingleSource.processElement(Read.java:1011)
   org.apache.beam.sdk.io.Read$OutputSingleSource$DoFnInvoker.invokeProcessElement(Unknown Source)
   org.apache.beam.fn.harness.FnApiDoFnRunner.processElementForParDo(FnApiDoFnRunner.java:773)
   org.apache.beam.fn.harness.data.PCollectionConsumerRegistry$MetricTrackingFnDataReceiver.accept(PCollectionConsumerRegistry.java:255)
   org.apache.beam.fn.harness.data.PCollectionConsumerRegistry$MetricTrackingFnDataReceiver.accept(PCollectionConsumerRegistry.java:209)
   org.apache.beam.fn.harness.BeamFnDataReadRunner.forwardElementToConsumer(BeamFnDataReadRunner.java:179)
   org.apache.beam.sdk.fn.data.BeamFnDataInboundObserver2.multiplexElements(BeamFnDataInboundObserver2.java:158)
   org.apache.beam.fn.harness.control.ProcessBundleHandler.processBundle(ProcessBundleHandler.java:515)
   org.apache.beam.fn.harness.control.BeamFnControlClient.delegateOnInstructionRequestType(BeamFnControlClient.java:151)
   org.apache.beam.fn.harness.control.BeamFnControlClient$InboundObserver.lambda$onNext$0(BeamFnControlClient.java:116)
   java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
   java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
   java.base/java.lang.Thread.run(Thread.java:833)
   Caused by: java.lang.NoSuchFieldError: EPOLL_DOMAIN_CLIENT_CHANNEL_TYPE
   io.grpc.netty.shaded.io.grpc.netty.UdsNettyChannelProvider.isAvailable(UdsNettyChannelProvider.java:34)
   io.grpc.ManagedChannelRegistry$ManagedChannelPriorityAccessor.isAvailable(ManagedChannelRegistry.java:172)
   io.grpc.ManagedChannelRegistry$ManagedChannelPriorityAccessor.isAvailable(ManagedChannelRegistry.java:168)
   io.grpc.ServiceProviders.loadAll(ServiceProviders.java:68)
   io.grpc.ManagedChannelRegistry.getDefaultRegistry(ManagedChannelRegistry.java:96)
   io.grpc.ManagedChannelProvider.provider(ManagedChannelProvider.java:41)
   io.grpc.ManagedChannelBuilder.forAddress(ManagedChannelBuilder.java:39)
   com.google.api.gax.grpc.InstantiatingGrpcChannelProvider.createSingleChannel(InstantiatingGrpcChannelProvider.java:357)
   com.google.api.gax.grpc.InstantiatingGrpcChannelProvider.access$1900(InstantiatingGrpcChannelProvider.java:82)
   com.google.api.gax.grpc.InstantiatingGrpcChannelProvider$1.createSingleChannel(InstantiatingGrpcChannelProvider.java:240)
   com.google.api.gax.grpc.ChannelPool.create(ChannelPool.java:72)
   com.google.api.gax.grpc.InstantiatingGrpcChannelProvider.createChannel(InstantiatingGrpcChannelProvider.java:250)
   com.google.api.gax.grpc.InstantiatingGrpcChannelProvider.getTransportChannel(InstantiatingGrpcChannelProvider.java:228)
   com.google.api.gax.rpc.ClientContext.create(ClientContext.java:241)
   com.google.cloud.bigquery.storage.v1.stub.GrpcBigQueryWriteStub.create(GrpcBigQueryWriteStub.java:132)
   com.google.cloud.bigquery.storage.v1.stub.BigQueryWriteStubSettings.createStub(BigQueryWriteStubSettings.java:145)
   com.google.cloud.bigquery.storage.v1.BigQueryWriteClient.<init>(BigQueryWriteClient.java:134)
   com.google.cloud.bigquery.storage.v1.BigQueryWriteClient.create(BigQueryWriteClient.java:116)
   org.apache.beam.sdk.io.gcp.bigquery.BigQueryServicesImpl.newBigQueryWriteClient(BigQueryServicesImpl.java:1401)
   org.apache.beam.sdk.io.gcp.bigquery.BigQueryServicesImpl.access$800(BigQueryServicesImpl.java:148)
   org.apache.beam.sdk.io.gcp.bigquery.BigQueryServicesImpl$DatasetServiceImpl.<init>(BigQueryServicesImpl.java:528)
   org.apache.beam.sdk.io.gcp.bigquery.BigQueryServicesImpl$DatasetServiceImpl.<init>(BigQueryServicesImpl.java:470)
   org.apache.beam.sdk.io.gcp.bigquery.BigQueryServicesImpl.getDatasetService(BigQueryServicesImpl.java:187)
   org.apache.beam.sdk.io.gcp.bigquery.BigQueryQueryHelper.executeQuery(BigQueryQueryHelper.java:99)
   org.apache.beam.sdk.io.gcp.bigquery.BigQueryQuerySourceDef.getTableReference(BigQueryQuerySourceDef.java:113)
   org.apache.beam.sdk.io.gcp.bigquery.BigQueryQuerySource.getTableToExtract(BigQueryQuerySource.java:64)
   org.apache.beam.sdk.io.gcp.bigquery.BigQuerySourceBase.extractFiles(BigQuerySourceBase.java:116)
   org.apache.beam.sdk.io.gcp.bigquery.BigQuerySourceBase.split(BigQuerySourceBase.java:156)
   org.apache.beam.sdk.io.Read$BoundedSourceAsSDFWrapperFn.splitRestriction(Read.java:292)
   ```
   
   I found workaround, fixing io.grpc:grpc-netty-shaded version to 1.46.0 solves this issue.
   ```gradle
   configurations.all {
       resolutionStrategy {
           force'io.grpc:grpc-netty-shaded:1.46.0'
       }
   }
   ```
   
   ### Issue Priority
   
   Priority: 2
   
   ### Issue Component
   
   Component: runner-dataflow


-- 
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.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] apilloud closed issue #23009: [Bug]: When io.grpc:grpc-netty-shaded:1.47.0>= is included in dependency, using DataflowRunnerV2 and BigQueryIO, exception thrown.

Posted by GitBox <gi...@apache.org>.
apilloud closed issue #23009: [Bug]: When io.grpc:grpc-netty-shaded:1.47.0>= is included in dependency, using DataflowRunnerV2 and BigQueryIO, exception thrown.
URL: https://github.com/apache/beam/issues/23009


-- 
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.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] takaaki7 commented on issue #23009: [Bug]: When io.grpc:grpc-netty-shaded:1.47.0>= is included in dependency, using DataflowRunnerV2 and BigQueryIO, exception thrown.

Posted by GitBox <gi...@apache.org>.
takaaki7 commented on issue #23009:
URL: https://github.com/apache/beam/issues/23009#issuecomment-1378204376

   Is there a way to install unreleased version `2.44.0`?
   I encountered other problem because of workaround (fixing io.grpc:grpc-netty-shaded to 1.46.0)... 
   Using latest bigtable client version `2.17.1`, failed to initialize by AbstractMethodError `Receiver class io.grpc.netty.shaded.io.grpc.netty.NettyChannelProvider does not define or inherit an implementation of the resolved method 'abstract java.util.Collection getSupportedSocketAddressTypes()' of abstract class io.grpc.ManagedChannelProvider."`


-- 
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.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] apilloud commented on issue #23009: [Bug]: When io.grpc:grpc-netty-shaded:1.47.0>= is included in dependency, using DataflowRunnerV2 and BigQueryIO, exception thrown.

Posted by GitBox <gi...@apache.org>.
apilloud commented on issue #23009:
URL: https://github.com/apache/beam/issues/23009#issuecomment-1381214504

   When I forced the version of io.grpc:grpc-netty-shaded to 1.48.0 I was able to trivially reproduce this (job fails only on dataflow with --experiments=use_runner_v2) on Beam 2.39.0 with a pipeline consisting of `pipeline.apply(BigQueryIO.readTableRows().from("clouddataflow-readonly:samples.weather_stations"));`. I was not able to get things into a state where the version of Grpc selected was something other than 1.44.0 without forcing that version, so there is probably more to that but it doesn't matter for fixing this.
   
   Updated to Beam 2.43.0 and forced grpc-netty-shaded to 1.44.0 or 1.46.0 but that didn't break BigQueryIO  (GRPC 1.49.2 is the version that was selected by default). It will take some more work to validate the fix.


-- 
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.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] kennknowles commented on issue #23009: [Bug]: When io.grpc:grpc-netty-shaded:1.47.0>= is included in dependency, using DataflowRunnerV2 and BigQueryIO, exception thrown.

Posted by GitBox <gi...@apache.org>.
kennknowles commented on issue #23009:
URL: https://github.com/apache/beam/issues/23009#issuecomment-1335958657

   @apilloud @lukecwik is this the thing that is now fixed?


-- 
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.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] apilloud commented on issue #23009: [Bug]: When io.grpc:grpc-netty-shaded:1.47.0>= is included in dependency, using DataflowRunnerV2 and BigQueryIO, exception thrown.

Posted by GitBox <gi...@apache.org>.
apilloud commented on issue #23009:
URL: https://github.com/apache/beam/issues/23009#issuecomment-1382383569

   I have confirmed this issue is fixed in 2.44.0. To test, I backported #24264 to Beam 2.39.0, built a new java container, and reran the test pipeline using the new container.


-- 
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.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] bmgandre commented on issue #23009: [Bug]: When io.grpc:grpc-netty-shaded:1.47.0>= is included in dependency, using DataflowRunnerV2 and BigQueryIO, exception thrown.

Posted by GitBox <gi...@apache.org>.
bmgandre commented on issue #23009:
URL: https://github.com/apache/beam/issues/23009#issuecomment-1343157784

   Fixing io.grpc:grpc-netty-shaded version to 1.46.0 didn't solve the issue for me.


-- 
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.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] apilloud commented on issue #23009: [Bug]: When io.grpc:grpc-netty-shaded:1.47.0>= is included in dependency, using DataflowRunnerV2 and BigQueryIO, exception thrown.

Posted by GitBox <gi...@apache.org>.
apilloud commented on issue #23009:
URL: https://github.com/apache/beam/issues/23009#issuecomment-1379072750

   You should be able to test 2.44.0 with the release candidate artifacts: https://lists.apache.org/thread/r3wbky2dmrl27sfy4vn4rytkj8bswovw


-- 
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.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] lukecwik commented on issue #23009: [Bug]: When io.grpc:grpc-netty-shaded:1.47.0>= is included in dependency, using DataflowRunnerV2 and BigQueryIO, exception thrown.

Posted by GitBox <gi...@apache.org>.
lukecwik commented on issue #23009:
URL: https://github.com/apache/beam/issues/23009#issuecomment-1340310728

   @apilloud was still verifying that this was indeed fixed with https://github.com/apache/beam/pull/24264


-- 
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.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org