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

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

                Author: ASF GitHub Bot
            Created on: 07/May/20 20:06
            Start Date: 07/May/20 20:06
    Worklog Time Spent: 10m 
      Work Description: lukecwik commented on a change in pull request #11596:
URL: https://github.com/apache/beam/pull/11596#discussion_r421756445



##########
File path: 
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/OrderedTimeRange.java
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.sdk.transforms.splittabledofn;
+
+import static 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.coders.AtomicCoder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.InstantCoder;
+import org.apache.beam.sdk.util.VarInt;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Objects;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+/** A restriction represented by a range of Instants [from, to). */
+public class OrderedTimeRange

Review comment:
       why not use the offset range tracker and convert time to `long`?

##########
File path: 
sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/OffsetRangeTracker.java
##########
@@ -94,6 +94,9 @@ public void checkDone() throws IllegalStateException {
     if (range.getFrom() == range.getTo()) {
       return;
     }
+    if (lastAttemptedOffset == null) {
+      throw new IllegalStateException("lastAttemptedOffset should not be 
null");
+    }
     checkState(
         lastAttemptedOffset >= range.getTo() - 1,
         "Last attempted offset was %s in range %s, claiming work in [%s, %s) 
was not attempted",

Review comment:
       Please make this a seperate PR and add a test that covers this case to 
the OffsetRangeTrackerTest
   
   ```suggestion
       checkState(
           lastAttemptedOffset != null && lastAttemptedOffset >= range.getTo() 
- 1,
           "Last attempted offset was %s in range %s, claiming work in [%s, %s) 
was not attempted",
   ```

##########
File path: 
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IO.java
##########
@@ -472,24 +548,120 @@ public void initClient() throws IOException {
       this.client = new HttpHealthcareApiClient();
     }
 
+    @GetInitialRestriction
+    public OrderedTimeRange getEarliestToLatestRestriction(@Element String 
hl7v2Store)
+        throws IOException {
+      from = this.client.getEarliestHL7v2SendTime(hl7v2Store, this.filter);
+      // filters are [from, to) to match logic of OffsetRangeTracker but need 
latest element to be
+      // included in results set to add an extra ms to the upper bound.
+      to = this.client.getLatestHL7v2SendTime(hl7v2Store, this.filter).plus(1);
+      return new OrderedTimeRange(from, to);
+    }
+
+    @NewTracker
+    public OrderedTimeRangeTracker newTracker(@Restriction OrderedTimeRange 
timeRange) {
+      return timeRange.newTracker();
+    }
+
+    @SplitRestriction
+    public void split(
+        @Restriction OrderedTimeRange timeRange, 
OutputReceiver<OrderedTimeRange> out) {
+      // TODO(jaketf) How to pick optimal values for desiredNumOffsetsPerSplit 
?

Review comment:
       Typically, doing one split for every 64mbs of output has been our 
guidance here in the past.
   
   Dynamic splitting is meant to fill in the gap if there is too little 
splitting or a specific restriction has a lot more data then other restrictions.




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

For queries about this service, please contact Infrastructure at:
[email protected]


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

    Worklog Id:     (was: 431878)
    Time Spent: 2.5h  (was: 2h 20m)

> HL7v2IO.ListHL7v2Messages should be refactored to support more parallelization
> ------------------------------------------------------------------------------
>
>                 Key: BEAM-9856
>                 URL: https://issues.apache.org/jira/browse/BEAM-9856
>             Project: Beam
>          Issue Type: Improvement
>          Components: io-java-gcp
>            Reporter: Jacob Ferriero
>            Assignee: Jacob Ferriero
>            Priority: Minor
>          Time Spent: 2.5h
>  Remaining Estimate: 0h
>
> Currently the List Messages API paginates through in a single ProcessElement 
> Call.
> However we could get a restriction based on createTime using Messages.List 
> filter and orderby.
>  
> This is inline with the future roadmap of  HL7v2 bulk export API becomes 
> available that should allow splitting on (e.g. create time dimension). 
> Leveraging this bulk export might be  a future optimization to explore.
>  
> This could take one of two forms:
> 1. dyanmically splitable via splitable DoFn (sexy, beam idiomatic: make 
> optimization the runner's problem, potentially unnecessarily complex for this 
> use case )
> 2. static splitting on some time partition e.g. finding the earliest 
> createTime and emitting a PCollection of 1 hour partitions and paginating 
> through each hour of data w/ in the time frame that the store spans, in a 
> separate ProcessElement. (easy to implement but will likely have hot keys / 
> stragglers based on "busy hours")
>  



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

Reply via email to