You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by "baeminbo (via GitHub)" <gi...@apache.org> on 2023/09/20 02:12:11 UTC

[GitHub] [beam] baeminbo opened a new pull request, #28548: [#20970] Fix gRPC leak by closing reader at WorkerCustomSources.UnboundedReaderIterator in Dataflow worker

baeminbo opened a new pull request, #28548:
URL: https://github.com/apache/beam/pull/28548

   This is another case of #20970 (harmless error logs with gRPC leak). 
   
   `BigQueryIO.read()` with `Method.DIRECT_READ` in Dataflow **streaming** jobs can cause Dataflow to output the gRPC leak error logs. From the stacktrace below, I believe the root cause is `UnboundedReaderIterator.close()` doesn't close the `reader`.
   
   
   ```
   java.lang.RuntimeException: ManagedChannel allocation site
   	at io.grpc.internal.ManagedChannelOrphanWrapper$ManagedChannelReference.<init>(ManagedChannelOrphanWrapper.java:102)
   	at io.grpc.internal.ManagedChannelOrphanWrapper.<init>(ManagedChannelOrphanWrapper.java:60)
   	at io.grpc.internal.ManagedChannelOrphanWrapper.<init>(ManagedChannelOrphanWrapper.java:51)
   	at io.grpc.internal.ManagedChannelImplBuilder.build(ManagedChannelImplBuilder.java:631)
   	at io.grpc.internal.AbstractManagedChannelImplBuilder.build(AbstractManagedChannelImplBuilder.java:297)
   	at com.google.api.gax.grpc.InstantiatingGrpcChannelProvider.createSingleChannel(InstantiatingGrpcChannelProvider.java:391)
   	at com.google.api.gax.grpc.ChannelPool.<init>(ChannelPool.java:107)
   	at com.google.api.gax.grpc.ChannelPool.create(ChannelPool.java:85)
   	at com.google.api.gax.grpc.InstantiatingGrpcChannelProvider.createChannel(InstantiatingGrpcChannelProvider.java:237)
   	at com.google.api.gax.grpc.InstantiatingGrpcChannelProvider.getTransportChannel(InstantiatingGrpcChannelProvider.java:231)
   	at com.google.api.gax.rpc.ClientContext.create(ClientContext.java:236)
   	at com.google.cloud.bigquery.storage.v1.stub.EnhancedBigQueryReadStub.create(EnhancedBigQueryReadStub.java:99)
   	at com.google.cloud.bigquery.storage.v1.BigQueryReadClient.<init>(BigQueryReadClient.java:130)
   	at com.google.cloud.bigquery.storage.v1.BigQueryReadClient.create(BigQueryReadClient.java:110)
   	at org.apache.beam.sdk.io.gcp.bigquery.BigQueryServicesImpl$StorageClientImpl.<init>(BigQueryServicesImpl.java:1667)
   	at org.apache.beam.sdk.io.gcp.bigquery.BigQueryServicesImpl$StorageClientImpl.<init>(BigQueryServicesImpl.java:1598)
   	at org.apache.beam.sdk.io.gcp.bigquery.BigQueryServicesImpl.getStorageClient(BigQueryServicesImpl.java:201)
   	at org.apache.beam.sdk.io.gcp.bigquery.BigQueryStorageStreamSource$BigQueryStorageStreamReader.<init>(BigQueryStorageStreamSource.java:194)
   	at org.apache.beam.sdk.io.gcp.bigquery.BigQueryStorageStreamSource$BigQueryStorageStreamReader.<init>(BigQueryStorageStreamSource.java:147)
   	at org.apache.beam.sdk.io.gcp.bigquery.BigQueryStorageStreamSource.createReader(BigQueryStorageStreamSource.java:138)
   	at org.apache.beam.sdk.io.gcp.bigquery.BigQueryStorageStreamSource.createReader(BigQueryStorageStreamSource.java:56)
   	at org.apache.beam.runners.core.construction.UnboundedReadFromBoundedSource$BoundedToUnboundedSourceAdapter$ResidualSource.advance(UnboundedReadFromBoundedSource.java:473)
   	at org.apache.beam.runners.core.construction.UnboundedReadFromBoundedSource$BoundedToUnboundedSourceAdapter$ResidualSource.access$300(UnboundedReadFromBoundedSource.java:452)
   	at org.apache.beam.runners.core.construction.UnboundedReadFromBoundedSource$BoundedToUnboundedSourceAdapter$Reader.advance(UnboundedReadFromBoundedSource.java:304)
   	at org.apache.beam.runners.core.construction.UnboundedReadFromBoundedSource$BoundedToUnboundedSourceAdapter$Reader.start(UnboundedReadFromBoundedSource.java:297)
   	at org.apache.beam.runners.dataflow.worker.WorkerCustomSources$UnboundedReaderIterator.start(WorkerCustomSources.java:816)
   	at org.apache.beam.runners.dataflow.worker.util.common.worker.ReadOperation$SynchronizedReaderIterator.start(ReadOperation.java:381)
   	at org.apache.beam.runners.dataflow.worker.util.common.worker.ReadOperation.runReadLoop(ReadOperation.java:211)
   	at org.apache.beam.runners.dataflow.worker.util.common.worker.ReadOperation.start(ReadOperation.java:169)
   	at org.apache.beam.runners.dataflow.worker.util.common.worker.MapTaskExecutor.execute(MapTaskExecutor.java:83)
   	at org.apache.beam.runners.dataflow.worker.StreamingDataflowWorker.process(StreamingDataflowWorker.java:1433)
   	at org.apache.beam.runners.dataflow.worker.StreamingDataflowWorker.access$800(StreamingDataflowWorker.java:155)
   	at org.apache.beam.runners.dataflow.worker.StreamingDataflowWorker$4.run(StreamingDataflowWorker.java:1056)
   	at org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor.lambda$executeLockHeld$0(BoundedQueueExecutor.java:163)
   	at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
   	at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
   	at java.base/java.lang.Thread.run(Thread.java:834)
   ```
   


-- 
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] github-actions[bot] commented on pull request #28548: [#20970] Fix gRPC leak by closing ResidualSource at BoundedToUnboundedSourceAdapter.Reader#init() in Dataflow worker

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #28548:
URL: https://github.com/apache/beam/pull/28548#issuecomment-1732585658

   Stopping reviewer notifications for this pull request: review requested by someone other than the bot, ceding control


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


Re: [PR] [#20970] Fix gRPC leak by closing ResidualSource at BoundedToUnboundedSourceAdapter.Reader#init() in Dataflow worker [beam]

Posted by "baeminbo (via GitHub)" <gi...@apache.org>.
baeminbo commented on PR #28548:
URL: https://github.com/apache/beam/pull/28548#issuecomment-1756615473

   Run Java PreCommit


-- 
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 pull request #28548: [#20970] Fix gRPC leak by closing ResidualSource at BoundedToUnboundedSourceAdapter.Reader#init() in Dataflow worker

Posted by "kennknowles (via GitHub)" <gi...@apache.org>.
kennknowles commented on PR #28548:
URL: https://github.com/apache/beam/pull/28548#issuecomment-1735491741

   I can see that testing a fix for a memory leak may be challenging, but can we do anything to test this change?


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


Re: [PR] [#20970] Fix gRPC leak by closing ResidualSource at BoundedToUnboundedSourceAdapter.Reader#init() in Dataflow worker [beam]

Posted by "baeminbo (via GitHub)" <gi...@apache.org>.
baeminbo commented on PR #28548:
URL: https://github.com/apache/beam/pull/28548#issuecomment-1756992641

   @kennknowles Some tests were flaky. But, finally, it passed all the checks. 


-- 
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] baeminbo commented on a diff in pull request #28548: [#20970] Fix gRPC leak by closing ResidualSource at BoundedToUnboundedSourceAdapter.Reader#init() in Dataflow worker

Posted by "baeminbo (via GitHub)" <gi...@apache.org>.
baeminbo commented on code in PR #28548:
URL: https://github.com/apache/beam/pull/28548#discussion_r1337975910


##########
runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java:
##########
@@ -288,6 +288,15 @@ private void init(
             residualElementsList == null
                 ? new ResidualElements(Collections.emptyList())
                 : new ResidualElements(residualElementsList);
+
+        if (this.residualSource != null) {

Review Comment:
   Yes, it's at [Reader.getCheckpointMark()](
   https://github.com/apache/beam/blob/204c69721a8d5ef98423ff03bea0b6ab90b10dc3/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java#L385). 
   
   The `getCheckpointMark()` is called at bundle finish in Dataflow streaming jobs, and `close()` will be pending as readers are cached. A reader can be reused in next bundles.



-- 
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] baeminbo commented on pull request #28548: [#20970] Fix gRPC leak by closing ResidualSource at BoundedToUnboundedSourceAdapter.Reader#init() in Dataflow worker

Posted by "baeminbo (via GitHub)" <gi...@apache.org>.
baeminbo commented on PR #28548:
URL: https://github.com/apache/beam/pull/28548#issuecomment-1732458804

   retest this please.


-- 
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] baeminbo commented on a diff in pull request #28548: [#20970] Fix gRPC leak by closing ResidualSource at BoundedToUnboundedSourceAdapter.Reader#init() in Dataflow worker

Posted by "baeminbo (via GitHub)" <gi...@apache.org>.
baeminbo commented on code in PR #28548:
URL: https://github.com/apache/beam/pull/28548#discussion_r1341036490


##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSources.java:
##########
@@ -776,7 +776,7 @@ public double getRemainingParallelism() {
 
   private static class UnboundedReaderIterator<T>
       extends NativeReader.NativeReaderIterator<WindowedValue<ValueWithRecordId<T>>> {
-    private final UnboundedSource.UnboundedReader<T> reader;
+    private final UnboundedSource.UnboundedReader<T> reader; // not owned

Review Comment:
   Thanks, I added a comment to explain it in detail, rather than a short ambiguous comment.



-- 
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] baeminbo commented on pull request #28548: [#20970] Fix gRPC leak by closing reader at WorkerCustomSources.UnboundedReaderIterator in Dataflow worker

Posted by "baeminbo (via GitHub)" <gi...@apache.org>.
baeminbo commented on PR #28548:
URL: https://github.com/apache/beam/pull/28548#issuecomment-1727130902

   retest this please


-- 
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 a diff in pull request #28548: [#20970] Fix gRPC leak by closing ResidualSource at BoundedToUnboundedSourceAdapter.Reader#init() in Dataflow worker

Posted by "kennknowles (via GitHub)" <gi...@apache.org>.
kennknowles commented on code in PR #28548:
URL: https://github.com/apache/beam/pull/28548#discussion_r1337164416


##########
runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java:
##########
@@ -288,6 +288,15 @@ private void init(
             residualElementsList == null
                 ? new ResidualElements(Collections.emptyList())
                 : new ResidualElements(residualElementsList);
+
+        if (this.residualSource != null) {

Review Comment:
   Just checking - this means that we are calling `init()` again without ever calling `close()` on the reader?



-- 
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 pull request #28548: [#20970] Fix gRPC leak by closing ResidualSource at BoundedToUnboundedSourceAdapter.Reader#init() in Dataflow worker

Posted by "kennknowles (via GitHub)" <gi...@apache.org>.
kennknowles commented on PR #28548:
URL: https://github.com/apache/beam/pull/28548#issuecomment-1741064238

   I feel like I want to run a bit of a variety of integration tests but I'm not 100% sure which ones.


-- 
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] github-actions[bot] commented on pull request #28548: [#20970] Fix gRPC leak by closing reader at WorkerCustomSources.UnboundedReaderIterator in Dataflow worker

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #28548:
URL: https://github.com/apache/beam/pull/28548#issuecomment-1726831725

   Checks are failing. Will not request review until checks are succeeding. If you'd like to override that behavior, comment `assign set of reviewers`


-- 
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] baeminbo commented on pull request #28548: [#20970] Fix gRPC leak by closing reader at WorkerCustomSources.UnboundedReaderIterator in Dataflow worker

Posted by "baeminbo (via GitHub)" <gi...@apache.org>.
baeminbo commented on PR #28548:
URL: https://github.com/apache/beam/pull/28548#issuecomment-1729050977

   retest this please


-- 
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] liferoad commented on pull request #28548: [#20970] Fix gRPC leak by closing ResidualSource at BoundedToUnboundedSourceAdapter.Reader#init() in Dataflow worker

Posted by "liferoad (via GitHub)" <gi...@apache.org>.
liferoad commented on PR #28548:
URL: https://github.com/apache/beam/pull/28548#issuecomment-1732585283

   R: @kennknowles 


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


Re: [PR] [#20970] Fix gRPC leak by closing ResidualSource at BoundedToUnboundedSourceAdapter.Reader#init() in Dataflow worker [beam]

Posted by "baeminbo (via GitHub)" <gi...@apache.org>.
baeminbo commented on PR #28548:
URL: https://github.com/apache/beam/pull/28548#issuecomment-1756760078

   Run Java_GCP_IO_Direct PreCommit


-- 
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] baeminbo commented on pull request #28548: [#20970] Fix gRPC leak by closing reader at WorkerCustomSources.UnboundedReaderIterator in Dataflow worker

Posted by "baeminbo (via GitHub)" <gi...@apache.org>.
baeminbo commented on PR #28548:
URL: https://github.com/apache/beam/pull/28548#issuecomment-1726958411

   retest this please


-- 
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 a diff in pull request #28548: [#20970] Fix gRPC leak by closing ResidualSource at BoundedToUnboundedSourceAdapter.Reader#init() in Dataflow worker

Posted by "kennknowles (via GitHub)" <gi...@apache.org>.
kennknowles commented on code in PR #28548:
URL: https://github.com/apache/beam/pull/28548#discussion_r1338653438


##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSources.java:
##########
@@ -776,7 +776,7 @@ public double getRemainingParallelism() {
 
   private static class UnboundedReaderIterator<T>
       extends NativeReader.NativeReaderIterator<WindowedValue<ValueWithRecordId<T>>> {
-    private final UnboundedSource.UnboundedReader<T> reader;
+    private final UnboundedSource.UnboundedReader<T> reader; // not owned

Review Comment:
   What does ownership mean here? Not to do with memory management obviously. If I understand correctly, it means that there may be other calls to various state methods so you can never assume what state the reader will be in? (it would be worth writing the whole long comment about what this means here)



##########
runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java:
##########
@@ -505,6 +514,7 @@ BoundedSource<T> getSource() {
       }
 
       Checkpoint<T> getCheckpointMark() {
+        checkArgument(!closed, "getCheckpointMark() call on closed %s", getClass().getName());

Review Comment:
   This should be `checkState` since there are no arguments.



##########
runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java:
##########
@@ -288,6 +288,15 @@ private void init(
             residualElementsList == null
                 ? new ResidualElements(Collections.emptyList())
                 : new ResidualElements(residualElementsList);
+
+        if (this.residualSource != null) {

Review Comment:
   Oh yea based on that the change is obviously correct. Thanks!



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


Re: [PR] [#20970] Fix gRPC leak by closing ResidualSource at BoundedToUnboundedSourceAdapter.Reader#init() in Dataflow worker [beam]

Posted by "kennknowles (via GitHub)" <gi...@apache.org>.
kennknowles merged PR #28548:
URL: https://github.com/apache/beam/pull/28548


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


Re: [PR] [#20970] Fix gRPC leak by closing ResidualSource at BoundedToUnboundedSourceAdapter.Reader#init() in Dataflow worker [beam]

Posted by "baeminbo (via GitHub)" <gi...@apache.org>.
baeminbo commented on PR #28548:
URL: https://github.com/apache/beam/pull/28548#issuecomment-1756863810

   Run Java_GCP_IO_Direct PreCommit


-- 
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] baeminbo commented on pull request #28548: [#20970] Fix gRPC leak by closing reader at WorkerCustomSources.UnboundedReaderIterator in Dataflow worker

Posted by "baeminbo (via GitHub)" <gi...@apache.org>.
baeminbo commented on PR #28548:
URL: https://github.com/apache/beam/pull/28548#issuecomment-1726943998

   retest this please


-- 
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] baeminbo commented on a diff in pull request #28548: [#20970] Fix gRPC leak by closing ResidualSource at BoundedToUnboundedSourceAdapter.Reader#init() in Dataflow worker

Posted by "baeminbo (via GitHub)" <gi...@apache.org>.
baeminbo commented on code in PR #28548:
URL: https://github.com/apache/beam/pull/28548#discussion_r1341034537


##########
runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java:
##########
@@ -505,6 +514,7 @@ BoundedSource<T> getSource() {
       }
 
       Checkpoint<T> getCheckpointMark() {
+        checkArgument(!closed, "getCheckpointMark() call on closed %s", getClass().getName());

Review Comment:
   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


Re: [PR] [#20970] Fix gRPC leak by closing ResidualSource at BoundedToUnboundedSourceAdapter.Reader#init() in Dataflow worker [beam]

Posted by "baeminbo (via GitHub)" <gi...@apache.org>.
baeminbo commented on PR #28548:
URL: https://github.com/apache/beam/pull/28548#issuecomment-1749926173

   Run Java_GCP_IO_Direct PreCommit


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


Re: [PR] [#20970] Fix gRPC leak by closing ResidualSource at BoundedToUnboundedSourceAdapter.Reader#init() in Dataflow worker [beam]

Posted by "baeminbo (via GitHub)" <gi...@apache.org>.
baeminbo commented on PR #28548:
URL: https://github.com/apache/beam/pull/28548#issuecomment-1756809749

   Run Java PreCommit


-- 
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 pull request #28548: [#20970] Fix gRPC leak by closing ResidualSource at BoundedToUnboundedSourceAdapter.Reader#init() in Dataflow worker

Posted by "kennknowles (via GitHub)" <gi...@apache.org>.
kennknowles commented on PR #28548:
URL: https://github.com/apache/beam/pull/28548#issuecomment-1741197706

   Run Java_IOs_Direct PreCommit


-- 
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] baeminbo commented on pull request #28548: [#20970] Fix gRPC leak by closing ResidualSource at BoundedToUnboundedSourceAdapter.Reader#init() in Dataflow worker

Posted by "baeminbo (via GitHub)" <gi...@apache.org>.
baeminbo commented on PR #28548:
URL: https://github.com/apache/beam/pull/28548#issuecomment-1740835178

   @kennknowles I added a unit test `UnboundedReadFromBoundedSourceTest#testReadersClosedProperly` to check reader closing leak. Could you review the change? Thank you.


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


Re: [PR] [#20970] Fix gRPC leak by closing ResidualSource at BoundedToUnboundedSourceAdapter.Reader#init() in Dataflow worker [beam]

Posted by "kennknowles (via GitHub)" <gi...@apache.org>.
kennknowles commented on PR #28548:
URL: https://github.com/apache/beam/pull/28548#issuecomment-1749459982

   OK the failure we see has been fixed at HEAD. Can you rebase against the master branch and we can re-run to get green.


-- 
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] baeminbo commented on pull request #28548: [#20970] Fix gRPC leak by closing ResidualSource at BoundedToUnboundedSourceAdapter.Reader#init() in Dataflow worker

Posted by "baeminbo (via GitHub)" <gi...@apache.org>.
baeminbo commented on PR #28548:
URL: https://github.com/apache/beam/pull/28548#issuecomment-1730908098

   I put a comment `not owned` to the `reader` field in the class `UnboundedReaderIterator` in `WorkerCustomSources.java`. My first fix was simply to close the `reader` in `UnboundedReaderIterator#close()`. But, it had a bug to read the same data again from a reader in integration tests.  To avoid this mistake again, I put the comment and additional check at `ResidualSource#getCheckpointMark()`


-- 
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] liferoad commented on pull request #28548: [#20970] Fix gRPC leak by closing ResidualSource at BoundedToUnboundedSourceAdapter.Reader#init() in Dataflow worker

Posted by "liferoad (via GitHub)" <gi...@apache.org>.
liferoad commented on PR #28548:
URL: https://github.com/apache/beam/pull/28548#issuecomment-1732585174

   Run Java_GCP_IO_Direct PreCommit


-- 
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] liferoad commented on pull request #28548: [#20970] Fix gRPC leak by closing ResidualSource at BoundedToUnboundedSourceAdapter.Reader#init() in Dataflow worker

Posted by "liferoad (via GitHub)" <gi...@apache.org>.
liferoad commented on PR #28548:
URL: https://github.com/apache/beam/pull/28548#issuecomment-1732585337

   R: @Abacn 


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