[ 
https://issues.apache.org/jira/browse/BEAM-6612?focusedWorklogId=197120&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-197120
 ]

ASF GitHub Bot logged work on BEAM-6612:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 11/Feb/19 18:53
            Start Date: 11/Feb/19 18:53
    Worklog Time Spent: 10m 
      Work Description: Ardagan commented on pull request #7764: [BEAM-6612] 
Modify QueueingBeamFnDataClient to become MetricsBeamFnDataClient and make all 
processElement() calls occur in parallel again.
URL: https://github.com/apache/beam/pull/7764#discussion_r255643911
 
 

 ##########
 File path: 
sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/MetricsBeamFnDataClient.java
 ##########
 @@ -90,36 +92,27 @@ private boolean allDone() {
    * <p>All {@link InboundDataClient}s will be failed if processing throws an 
exception.
    *
    * <p>This method is NOT thread safe. This should only be invoked by a 
single thread, and is
-   * intended for use with a newly constructed QueueingBeamFnDataClient in 
{@link
+   * intended for use with a newly constructed MetricsBeamFnDataClient in 
{@link
    * ProcessBundleHandler#processBundle(InstructionRequest)}.
    */
-  public void drainAndBlock() throws Exception {
+  public void waitTillDone() throws Exception {
     while (true) {
-      try {
-        ConsumerAndData tuple = queue.poll(200, TimeUnit.MILLISECONDS);
-        if (tuple != null) {
-          // Forward to the consumers who cares about this data.
-          tuple.consumer.accept(tuple.data);
-        } else {
-          // Note: We do not expect to ever hit this point without receiving 
all values
-          // as (1) The InboundObserver will not be set to Done until the
-          // QueuingFnDataReceiver.accept() call returns and will not be 
invoked again.
-          // (2) The QueueingFnDataReceiver will not return until the value is 
received in
-          // drainAndBlock, because of the use of the SynchronousQueue.
-          if (allDone()) {
-            break;
-          }
-        }
-      } catch (Exception e) {
-        LOG.error("Client failed to dequeue and process WindowedValue", e);
-        for (InboundDataClient inboundDataClient : 
inboundDataClients.keySet()) {
-          inboundDataClient.fail(e);
-        }
-        throw e;
+      if (allDone()) {
 
 Review comment:
   General idea:
   we can make allDone return future and wait for it.
 
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


Issue Time Tracking
-------------------

    Worklog Id:     (was: 197120)
    Time Spent: 50m  (was: 40m)

> Remove QueueingBeamFnDataClient
> -------------------------------
>
>                 Key: BEAM-6612
>                 URL: https://issues.apache.org/jira/browse/BEAM-6612
>             Project: Beam
>          Issue Type: New Feature
>          Components: java-fn-execution
>            Reporter: Alex Amato
>            Assignee: Alex Amato
>            Priority: Major
>              Labels: triaged
>          Time Spent: 50m
>  Remaining Estimate: 0h
>
> Remove QueueingBeamFnDataClient, which made process() calls all run on the 
> same thread.
> [~lcwik] and I came up with this design thinking that it was required to 
> process the bundle in parallel anyways, and we would have good performance. 
> However after speaking to Ken, there is no requirement for a bundle or key to 
> be processed in parallel. Elements are either iterables or single elements 
> which defines the needs for processing a group of elements on the same thread.
> Simply performing this change will lead to the following issues:
> (1) MetricsContainerImpl and MetricsContainer are not thread safe, so when 
> the process() functions enter the metric container context, they will be 
> accessing an thread-unsafe collection in parallel
> (2) An ExecutionStateTracker will be needed in every thread, So we will need 
> to
> create an instance and activate it in every GrpC thread which receives a new 
> element.
> (Will this get sampled properly, since the trackers will be short lived).
> (3) The SimpleExecutionStates being used will need to be thread safe as well? 
> I don't think so, because I don't think that the ExecutionStateSampler 
> invokes them in parallel.
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to