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 2023/01/17 06:26:35 UTC

[GitHub] [beam] alnzng opened a new pull request, #25031: Allow to set timeout for finishing a remote bundle in Samza portable runner

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

   addresses #25030 
   
   We have observed that some UDFs(with bugs or imported some bad third-party libs) could hang the runner and SDK harness processes in Samza portable mode. Both runner and SDK harness processes are in a zombie state and have no data processing in this case, however, the processes are not able to be shut down although Samza runner provides some built-in timeout functionality.
   
   This change will allow the users to set a timeout for closing a remote bundle in Samza portable runner. Once the timeout occurred, the runner and SDK processes will be shut down probably by Samza's built-in task timeout functionality if enabled.
   ------------------------
   
   Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:
   
    - [ ] Mention the appropriate issue in your description (for example: `addresses #123`), if applicable. This will automatically add a link to the pull request in the issue. If you would like the issue to automatically close on merging the pull request, comment `fixes #<ISSUE NUMBER>` instead.
    - [ ] Update `CHANGES.md` with noteworthy changes.
    - [ ] If this contribution is large, please file an Apache [Individual Contributor License Agreement](https://www.apache.org/licenses/icla.pdf).
   
   See the [Contributor Guide](https://beam.apache.org/contribute) for more tips on [how to make review process smoother](https://beam.apache.org/contribute/get-started-contributing/#make-the-reviewers-job-easier).
   
   To check the build health, please visit [https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md](https://github.com/apache/beam/blob/master/.test-infra/BUILD_STATUS.md)
   
   GitHub Actions Tests Status (on master branch)
   ------------------------------------------------------------------------------------------------
   [![Build python source distribution and wheels](https://github.com/apache/beam/workflows/Build%20python%20source%20distribution%20and%20wheels/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Build+python+source+distribution+and+wheels%22+branch%3Amaster+event%3Aschedule)
   [![Python tests](https://github.com/apache/beam/workflows/Python%20tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Python+Tests%22+branch%3Amaster+event%3Aschedule)
   [![Java tests](https://github.com/apache/beam/workflows/Java%20Tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Java+Tests%22+branch%3Amaster+event%3Aschedule)
   [![Go tests](https://github.com/apache/beam/workflows/Go%20tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Go+tests%22+branch%3Amaster+event%3Aschedule)
   
   See [CI.md](https://github.com/apache/beam/blob/master/CI.md) for more information about GitHub Actions CI.
   


-- 
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] alnzng commented on a diff in pull request #25031: Allow to set timeout for finishing a remote bundle in Samza portable runner

Posted by GitBox <gi...@apache.org>.
alnzng commented on code in PR #25031:
URL: https://github.com/apache/beam/pull/25031#discussion_r1081971707


##########
runners/samza/src/main/java/org/apache/beam/runners/samza/util/RunWithTimeout.java:
##########
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.samza.util;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+public class RunWithTimeout {

Review Comment:
   Thanks for the review. Yep, I added it into SamzaDoFnRunners. @mynameborat was suggesting the same thing.



-- 
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] alnzng commented on a diff in pull request #25031: Allow to set timeout for finishing a remote bundle in Samza portable runner

Posted by GitBox <gi...@apache.org>.
alnzng commented on code in PR #25031:
URL: https://github.com/apache/beam/pull/25031#discussion_r1072645256


##########
runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPortablePipelineOptions.java:
##########
@@ -29,4 +30,11 @@
   String getFsTokenPath();
 
   void setFsTokenPath(String path);
+
+  @Description(
+      "Wait if necessary for completing a remote bundle processing for at most the given time (in milliseconds). if the value of timeout is negative, wait forever until the bundle processing is completed. Used only in portable mode.")
+  @Default.Long(-1)
+  long getBundleProcessingTimeout();
+
+  void setBundleProcessingTimeout(long timeoutMs);

Review Comment:
   Good suggestion. I think we can move this config to `SamzaPipelineOptions` and call out that this config is supported in portable mode only for now.



-- 
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] xinyuiscool merged pull request #25031: Allow to set timeout for finishing a remote bundle in Samza portable runner

Posted by GitBox <gi...@apache.org>.
xinyuiscool merged PR #25031:
URL: https://github.com/apache/beam/pull/25031


-- 
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] alnzng commented on a diff in pull request #25031: Allow to set timeout for finishing a remote bundle in Samza portable runner

Posted by GitBox <gi...@apache.org>.
alnzng commented on code in PR #25031:
URL: https://github.com/apache/beam/pull/25031#discussion_r1072645256


##########
runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPortablePipelineOptions.java:
##########
@@ -29,4 +30,11 @@
   String getFsTokenPath();
 
   void setFsTokenPath(String path);
+
+  @Description(
+      "Wait if necessary for completing a remote bundle processing for at most the given time (in milliseconds). if the value of timeout is negative, wait forever until the bundle processing is completed. Used only in portable mode.")
+  @Default.Long(-1)
+  long getBundleProcessingTimeout();
+
+  void setBundleProcessingTimeout(long timeoutMs);

Review Comment:
   Good suggestion. I think we can move this config to `SamzaPipelineOptions` and call out that this config is supported in portable mode only.



##########
runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaDoFnRunners.java:
##########
@@ -439,6 +445,25 @@ public void finishBundle() {
       }
     }
 
+    private void closeBundle() throws Exception {
+      long bundleProcessingTimeout = pipelineOptions.getBundleProcessingTimeout();
+      if (bundleProcessingTimeout < 0) {
+        // RemoteBundle close blocks until all results are received
+        remoteBundle.close();
+      } else {
+        CompletableFuture<Void> future =
+            CompletableFuture.runAsync(
+                () -> {
+                  try {
+                    remoteBundle.close();
+                  } catch (Exception e) {
+                    throw new RuntimeException(e);
+                  }
+                });
+        future.get(bundleProcessingTimeout, TimeUnit.MILLISECONDS);

Review Comment:
   I didn't do that because of:
   1. The "remoteBundle.close()" method throws the "Exception" anyway. It seems we can't throw these concrete exceptions(e.g. TimeoutException) in the method signature.
   2. the current `finishBundle()` implementation just simply catches all exceptions without differentiating the concrete exceptions
   
   



##########
runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaDoFnRunners.java:
##########
@@ -439,6 +445,25 @@ public void finishBundle() {
       }
     }
 
+    private void closeBundle() throws Exception {
+      long bundleProcessingTimeout = pipelineOptions.getBundleProcessingTimeout();
+      if (bundleProcessingTimeout < 0) {
+        // RemoteBundle close blocks until all results are received
+        remoteBundle.close();
+      } else {
+        CompletableFuture<Void> future =
+            CompletableFuture.runAsync(
+                () -> {
+                  try {
+                    remoteBundle.close();
+                  } catch (Exception e) {
+                    throw new RuntimeException(e);

Review Comment:
   The `close()` method throws the checked exception `Exception`, we have to handle the checked exception explicitly in the Java lamda expression.



-- 
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] alnzng commented on pull request #25031: Allow to set timeout for finishing a remote bundle in Samza portable runner

Posted by GitBox <gi...@apache.org>.
alnzng commented on PR #25031:
URL: https://github.com/apache/beam/pull/25031#issuecomment-1385861659

   > Can we add unit tests?
   
   I didn't do that because there is no test cases created for this `SamzaDoFnRunners`. But I can do it for this newly added method only.


-- 
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] alnzng commented on a diff in pull request #25031: Allow to set timeout for finishing a remote bundle in Samza portable runner

Posted by GitBox <gi...@apache.org>.
alnzng commented on code in PR #25031:
URL: https://github.com/apache/beam/pull/25031#discussion_r1073148188


##########
runners/samza/src/main/java/org/apache/beam/runners/samza/util/RunWithTimeout.java:
##########
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.samza.util;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+public class RunWithTimeout {

Review Comment:
   Yes, they are similar. But the major difference is that the workflow to handle the timeout value is negative. For the native timeout value, the `CompletableFuture ` throws TimeoutException directly but in our case, the function call will wait forever until the function is completed.
   
   I would think it could benefit more that we keep it outside as a general util method since we might want to use it when we implement similar timeout logic for classic mode.
   
   WDYT?
   



-- 
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] xinyuiscool commented on a diff in pull request #25031: Allow to set timeout for finishing a remote bundle in Samza portable runner

Posted by GitBox <gi...@apache.org>.
xinyuiscool commented on code in PR #25031:
URL: https://github.com/apache/beam/pull/25031#discussion_r1081902350


##########
runners/samza/src/main/java/org/apache/beam/runners/samza/util/RunWithTimeout.java:
##########
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.samza.util;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+public class RunWithTimeout {

Review Comment:
   @alnzng : seems this util class is very small so might not worth adding it. I suggest inline it directly inside SamzaDoFnRunners. 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


[GitHub] [beam] alnzng commented on pull request #25031: Allow to set timeout for finishing a remote bundle in Samza portable runner

Posted by GitBox <gi...@apache.org>.
alnzng commented on PR #25031:
URL: https://github.com/apache/beam/pull/25031#issuecomment-1386128585

   @mynameborat Thanks for helping review. I addressed your comments, let me know if you have other questions.


-- 
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] mynameborat commented on a diff in pull request #25031: Allow to set timeout for finishing a remote bundle in Samza portable runner

Posted by GitBox <gi...@apache.org>.
mynameborat commented on code in PR #25031:
URL: https://github.com/apache/beam/pull/25031#discussion_r1072605173


##########
runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaDoFnRunners.java:
##########
@@ -439,6 +445,25 @@ public void finishBundle() {
       }
     }
 
+    private void closeBundle() throws Exception {
+      long bundleProcessingTimeout = pipelineOptions.getBundleProcessingTimeout();
+      if (bundleProcessingTimeout < 0) {
+        // RemoteBundle close blocks until all results are received
+        remoteBundle.close();
+      } else {
+        CompletableFuture<Void> future =
+            CompletableFuture.runAsync(
+                () -> {
+                  try {
+                    remoteBundle.close();
+                  } catch (Exception e) {
+                    throw new RuntimeException(e);

Review Comment:
   Why not throw the same exception instead of wrapping it up in `RuntimeException`? 



##########
runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPortablePipelineOptions.java:
##########
@@ -29,4 +30,11 @@
   String getFsTokenPath();
 
   void setFsTokenPath(String path);
+
+  @Description(
+      "Wait if necessary for completing a remote bundle processing for at most the given time (in milliseconds). if the value of timeout is negative, wait forever until the bundle processing is completed. Used only in portable mode.")
+  @Default.Long(-1)
+  long getBundleProcessingTimeout();
+
+  void setBundleProcessingTimeout(long timeoutMs);

Review Comment:
   Why is this specific to the portable mode? Since we support bundling in classic mode, we should support this across the two modes given users can perform logic within classic mode as well that can timeout (e.g., async processing)



##########
runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPortablePipelineOptions.java:
##########
@@ -29,4 +30,11 @@
   String getFsTokenPath();
 
   void setFsTokenPath(String path);
+
+  @Description(
+      "Wait if necessary for completing a remote bundle processing for at most the given time (in milliseconds). if the value of timeout is negative, wait forever until the bundle processing is completed. Used only in portable mode.")
+  @Default.Long(-1)
+  long getBundleProcessingTimeout();
+
+  void setBundleProcessingTimeout(long timeoutMs);

Review Comment:
   We can start with supporting the timeout in portable mode in this PR and keep the classic mode implementation in another PR



##########
runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaDoFnRunners.java:
##########
@@ -439,6 +445,25 @@ public void finishBundle() {
       }
     }
 
+    private void closeBundle() throws Exception {
+      long bundleProcessingTimeout = pipelineOptions.getBundleProcessingTimeout();
+      if (bundleProcessingTimeout < 0) {
+        // RemoteBundle close blocks until all results are received
+        remoteBundle.close();
+      } else {
+        CompletableFuture<Void> future =
+            CompletableFuture.runAsync(
+                () -> {
+                  try {
+                    remoteBundle.close();
+                  } catch (Exception e) {
+                    throw new RuntimeException(e);
+                  }
+                });
+        future.get(bundleProcessingTimeout, TimeUnit.MILLISECONDS);

Review Comment:
   This would throw `TimeoutException`/`ExecutionException` and mask the original cause in case of latter.
   I'd preferably bubble up the original cause so that we know what is wrong in case of failure.



-- 
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] alnzng commented on pull request #25031: Allow to set timeout for finishing a remote bundle in Samza portable runner

Posted by GitBox <gi...@apache.org>.
alnzng commented on PR #25031:
URL: https://github.com/apache/beam/pull/25031#issuecomment-1384893101

   @xinyuiscool please help take a look, 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


[GitHub] [beam] mynameborat commented on a diff in pull request #25031: Allow to set timeout for finishing a remote bundle in Samza portable runner

Posted by GitBox <gi...@apache.org>.
mynameborat commented on code in PR #25031:
URL: https://github.com/apache/beam/pull/25031#discussion_r1072888077


##########
runners/samza/src/main/java/org/apache/beam/runners/samza/util/RunWithTimeout.java:
##########
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.samza.util;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+public class RunWithTimeout {

Review Comment:
   Consider keeping this within `SamzaDoFnRunners` instead of introducing new class as this API isn't any different to `CompletableFuture` API nor the `ExecutorService`



##########
runners/samza/src/test/java/org/apache/beam/runners/samza/util/RunWithTimeoutTest.java:
##########
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.samza.util;
+
+import java.util.concurrent.TimeoutException;
+import org.junit.Test;
+
+public class RunWithTimeoutTest {

Review Comment:
   Add another test to capture the scenario where runnable throws an exception (mimicking the user code throwing an exception) and ensure if it is bubbled up?
   
   Looks like the code still doesn't bubble up the original cause but throws `ExecutionException` instead.



-- 
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] alnzng commented on a diff in pull request #25031: Allow to set timeout for finishing a remote bundle in Samza portable runner

Posted by GitBox <gi...@apache.org>.
alnzng commented on code in PR #25031:
URL: https://github.com/apache/beam/pull/25031#discussion_r1073148456


##########
runners/samza/src/test/java/org/apache/beam/runners/samza/util/RunWithTimeoutTest.java:
##########
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.samza.util;
+
+import java.util.concurrent.TimeoutException;
+import org.junit.Test;
+
+public class RunWithTimeoutTest {

Review Comment:
   Added a new test, can you please check if that is what you wanted to verify?



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