You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@beam.apache.org by "Minbo Bae (Jira)" <ji...@apache.org> on 2021/05/18 04:10:00 UTC

[jira] [Updated] (BEAM-12356) BigQueryWriteClient in DatasetServiceImpl is not closed, which causes "ManagedChannel allocation site" exceptions

     [ https://issues.apache.org/jira/browse/BEAM-12356?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Minbo Bae updated BEAM-12356:
-----------------------------
    Attachment: bigquery_grpc.log

> BigQueryWriteClient in DatasetServiceImpl is not closed, which causes "ManagedChannel allocation site" exceptions
> -----------------------------------------------------------------------------------------------------------------
>
>                 Key: BEAM-12356
>                 URL: https://issues.apache.org/jira/browse/BEAM-12356
>             Project: Beam
>          Issue Type: Bug
>          Components: io-java-gcp
>    Affects Versions: 2.29.0
>            Reporter: Minbo Bae
>            Priority: P2
>         Attachments: bigquery_grpc.log
>
>
> [BigQueryWriteClient|https://github.com/apache/beam/blob/v2.29.0/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java#L461] in DatasetServiceImpl (added at [https://github.com/apache/beam/pull/14309)] is not closed.  This causes the error logs below in gRPC orphan channel clean up. I don't think this issue affect pipeline runs except the error logs, but could you take a look at that?
> A similar issue is reported for {{CloudBigtableIO}} at [https://github.com/googleapis/java-bigtable-hbase/issues/2658]
>  
> Logs extracted from GCP Dataflow 
> {quote}jsonPayload": {
>  "stage": "S0",
>  "step": “<REDACTED>/StreamingInserts/StreamingWriteTables/StreamingWrite/BatchedStreamingWrite.ViaBundleFinalization/ParMultiDo(BatchAndInsertElements)",
>  "work": “<REDACTED>,
>  "job": "<REDACTED>",
>  "worker": "<REDACTED>",
>  "exception": "java.lang.RuntimeException: ManagedChannel allocation site\n\tat io.grpc.internal.ManagedChannelOrphanWrapper$ManagedChannelReference.<init>(ManagedChannelOrphanWrapper.java:93)\n\tat io.grpc.internal.ManagedChannelOrphanWrapper.<init>(ManagedChannelOrphanWrapper.java:53)\n\tat io.grpc.internal.ManagedChannelOrphanWrapper.<init>(ManagedChannelOrphanWrapper.java:44)\n\tat io.grpc.internal.ManagedChannelImplBuilder.build(ManagedChannelImplBuilder.java:612)\n\tat io.grpc.internal.AbstractManagedChannelImplBuilder.build(AbstractManagedChannelImplBuilder.java:261)\n\tat com.google.api.gax.grpc.InstantiatingGrpcChannelProvider.createSingleChannel(InstantiatingGrpcChannelProvider.java:340)\n\tat com.google.api.gax.grpc.InstantiatingGrpcChannelProvider.access$1600(InstantiatingGrpcChannelProvider.java:73)\n\tat com.google.api.gax.grpc.InstantiatingGrpcChannelProvider$1.createSingleChannel(InstantiatingGrpcChannelProvider.java:214)\n\tat com.google.api.gax.grpc.ChannelPool.create(ChannelPool.java:72)\n\tat com.google.api.gax.grpc.InstantiatingGrpcChannelProvider.createChannel(InstantiatingGrpcChannelProvider.java:221)\n\tat com.google.api.gax.grpc.InstantiatingGrpcChannelProvider.getTransportChannel(InstantiatingGrpcChannelProvider.java:204)\n\tat com.google.api.gax.rpc.ClientContext.create(ClientContext.java:169)\n\tat com.google.cloud.bigquery.storage.v1beta2.stub.GrpcBigQueryWriteStub.create(GrpcBigQueryWriteStub.java:136)\n\tat com.google.cloud.bigquery.storage.v1beta2.stub.BigQueryWriteStubSettings.createStub(BigQueryWriteStubSettings.java:145)\n\tat com.google.cloud.bigquery.storage.v1beta2.BigQueryWriteClient.<init>(BigQueryWriteClient.java:120)\n\tat com.google.cloud.bigquery.storage.v1beta2.BigQueryWriteClient.create(BigQueryWriteClient.java:101)\n\tat org.apache.beam.sdk.io.gcp.bigquery.BigQueryServicesImpl.newBigQueryWriteClient(BigQueryServicesImpl.java:1255)\n\tat org.apache.beam.sdk.io.gcp.bigquery.BigQueryServicesImpl.access$800(BigQueryServicesImpl.java:135)\n\tat org.apache.beam.sdk.io.gcp.bigquery.BigQueryServicesImpl$DatasetServiceImpl.<init>(BigQueryServicesImpl.java:521)\n\tat org.apache.beam.sdk.io.gcp.bigquery.BigQueryServicesImpl$DatasetServiceImpl.<init>(BigQueryServicesImpl.java:449)\n\tat org.apache.beam.sdk.io.gcp.bigquery.BigQueryServicesImpl.getDatasetService(BigQueryServicesImpl.java:169)\n\tat org.apache.beam.sdk.io.gcp.bigquery.BatchedStreamingWrite.flushRows(BatchedStreamingWrite.java:374)\n\tat org.apache.beam.sdk.io.gcp.bigquery.BatchedStreamingWrite.access$800(BatchedStreamingWrite.java:69)\n\tat org.apache.beam.sdk.io.gcp.bigquery.BatchedStreamingWrite$BatchAndInsertElements.finishBundle(BatchedStreamingWrite.java:271)\n\tat org.apache.beam.sdk.io.gcp.bigquery.BatchedStreamingWrite$BatchAndInsertElements$DoFnInvoker.invokeFinishBundle(Unknown Source)\n\tat org.apache.beam.runners.dataflow.worker.repackaged.org.apache.beam.runners.core.SimpleDoFnRunner.finishBundle(SimpleDoFnRunner.java:242)\n\tat org.apache.beam.runners.dataflow.worker.SimpleParDoFn.finishBundle(SimpleParDoFn.java:432)\n\tat org.apache.beam.runners.dataflow.worker.util.common.worker.ParDoOperation.finish(ParDoOperation.java:56)\n\tat org.apache.beam.runners.dataflow.worker.util.common.worker.MapTaskExecutor.execute(MapTaskExecutor.java:103)\n\tat org.apache.beam.runners.dataflow.worker.StreamingDataflowWorker.process(StreamingDataflowWorker.java:1430)\n\tat org.apache.beam.runners.dataflow.worker.StreamingDataflowWorker.access$1100(StreamingDataflowWorker.java:165)\n\tat org.apache.beam.runners.dataflow.worker.StreamingDataflowWorker$7.run(StreamingDataflowWorker.java:1109)\n\tat java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)\n\tat java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)\n\tat java.base/java.lang.Thread.run(Thread.java:834)\n",
>  "message": "*~*~*~ Channel ManagedChannelImpl\{logId=69, target=bigquerystorage.googleapis.com:443} was not shutdown properly!!! ~*~*~*\n Make sure to call shutdown()/shutdownNow() and wait until awaitTermination() returns true.",
>  "thread": "446",
>  "logger": "io.grpc.internal.ManagedChannelOrphanWrapper"
>  },
> {quote}



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