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

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

                Author: ASF GitHub Bot
            Created on: 14/Aug/19 14:29
            Start Date: 14/Aug/19 14:29
    Worklog Time Spent: 10m 
      Work Description: je-ik commented on pull request #9032: [BEAM-7386] 
Bi-Temporal Join
URL: https://github.com/apache/beam/pull/9032#discussion_r313899337
 
 

 ##########
 File path: 
sdks/java/extensions/timeseries/src/main/java/org/apache/beam/sdk/extensions/timeseries/joins/BiTemporalStreams.java
 ##########
 @@ -0,0 +1,464 @@
+/*
+ * 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.extensions.timeseries.joins;
+
+import java.util.*;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.VarLongCoder;
+import org.apache.beam.sdk.state.*;
+import org.apache.beam.sdk.state.Timer;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Flatten;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.*;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This sample will take two streams left stream and right stream. It will 
match the left stream to
+ * the nearest right stream based on their timestamp. Nearest left stream is 
either <= to the right
+ * stream timestamp.
+ *
+ * <p>If two values in the right steam have the same timestamp then the 
results are
+ * non-deterministic.
+ */
+@Experimental
+public class BiTemporalStreams {
+
+  public static <K, V1, V2> BiTemporalJoin<K, V1, V2> join(
+      TupleTag<V1> leftTag, TupleTag<V2> rightTag, Duration window) {
+    return new BiTemporalJoin<>(leftTag, rightTag, window);
+  }
+
+  @Experimental
+  public static class BiTemporalJoin<K, V1, V2>
+      extends PTransform<KeyedPCollectionTuple<K>, 
PCollection<BiTemporalJoinResult<K, V1, V2>>> {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(BiTemporalStreams.class);
+
+    // Sets the limit at which point the processed left stream values are 
garbage collected
+    static int LEFT_STREAM_GC_LIMIT = 1000;
+
+    Coder<V1> leftCoder;
+    Coder<V2> rightCoder;
+
+    TupleTag<V1> leftTag;
+    TupleTag<V2> rightTag;
+
+    Duration window;
+
+    private BiTemporalJoin(TupleTag<V1> leftTag, TupleTag<V2> rightTag, 
Duration window) {
+      this.leftTag = leftTag;
+      this.rightTag = rightTag;
+      this.window = window;
+    }
+
+    public static <K, V1, V2> BiTemporalJoin create(
+        TupleTag<V1> leftTag, TupleTag<V2> rightTag, Duration window) {
+      return new BiTemporalJoin<K, V1, V2>(leftTag, rightTag, window);
+    }
+
+    public BiTemporalJoin setGCLimit(int gcLimit) {
+      LEFT_STREAM_GC_LIMIT = gcLimit;
+      return this;
+    }
+
+    @Override
+    public PCollection<BiTemporalJoinResult<K, V1, V2>> 
expand(KeyedPCollectionTuple<K> input) {
+
+      List<KeyedPCollectionTuple.TaggedKeyedPCollection<K, ?>> collections =
+          input.getKeyedCollections();
+
+      PCollection<KV<K, V1>> leftCollection = null;
+      PCollection<KV<K, V2>> rightCollection = null;
+
+      for (KeyedPCollectionTuple.TaggedKeyedPCollection<K, ?> t : collections) 
{
+
+        if (t.getTupleTag().equals(leftTag)) {
+          leftCollection =
+              ((KeyedPCollectionTuple.TaggedKeyedPCollection<K, V1>) 
t).getCollection();
+        }
+
+        if (t.getTupleTag().equals(rightTag)) {
+          rightCollection =
+              ((KeyedPCollectionTuple.TaggedKeyedPCollection<K, V2>) 
t).getCollection();
+        }
+      }
+
+      leftCoder = ((KvCoder<K, V1>) leftCollection.getCoder()).getValueCoder();
+      rightCoder = ((KvCoder<K, V2>) 
rightCollection.getCoder()).getValueCoder();
+
+      BiTemporalJoinResultCoder<K, V1, V2> biStreamJoinResultCoder =
+          new BiTemporalJoinResultCoder(input.getKeyCoder(), leftCoder, 
rightCoder);
+
+      PCollectionList<KV<K, BiTemporalJoinResult<K, V1, V2>>> l =
+          PCollectionList.of(
+                  leftCollection
+                      .apply(ParDo.of(new CreateStreamDataLeft<K, V1, V2>()))
+                      .setCoder(KvCoder.of(input.getKeyCoder(), 
biStreamJoinResultCoder)))
+              .and(
+                  rightCollection
+                      .apply(ParDo.of(new CreateStreamDataRight<K, V1, V2>()))
+                      .setCoder(KvCoder.of(input.getKeyCoder(), 
biStreamJoinResultCoder)));
+
+      return l.apply(Flatten.pCollections())
+          .apply(Window.into(FixedWindows.of(window)))
+          .apply(ParDo.of(new StreamMatcher<>(input.getKeyCoder(), leftCoder, 
rightCoder)));
+    }
+
+    public static class CreateStreamDataLeft<K, V1, V2>
+        extends DoFn<KV<K, V1>, KV<K, BiTemporalJoinResult<K, V1, V2>>> {
+
+      public static <K, V1, V2> CreateStreamDataLeft create() {
+        return new CreateStreamDataLeft<K, V1, V2>();
+      }
+
+      @ProcessElement
+      public void process(
+          ProcessContext c, OutputReceiver<KV<K, BiTemporalJoinResult<K, V1, 
V2>>> o) {
+
+        o.output(
+            KV.of(
+                c.element().getKey(),
+                new BiTemporalJoinResult<K, V1, V2>().setLeftData(c.element(), 
c.timestamp())));
+      }
+    }
+
+    public static class CreateStreamDataRight<K, V1, V2>
+        extends DoFn<KV<K, V2>, KV<K, BiTemporalJoinResult<K, V1, V2>>> {
+
+      public static <K, V1, V2> CreateStreamDataRight create() {
+        return new CreateStreamDataRight<K, V1, V2>();
+      }
+
+      @ProcessElement
+      public void process(
+          ProcessContext c, OutputReceiver<KV<K, BiTemporalJoinResult<K, V1, 
V2>>> o) {
+        o.output(
+            KV.of(
+                c.element().getKey(),
+                new BiTemporalJoinResult<K, V1, 
V2>().setRightData(c.element(), c.timestamp())));
+      }
+    }
+
+    public static class StreamMatcher<K, V1, V2>
+        extends DoFn<KV<K, BiTemporalJoinResult<K, V1, V2>>, 
BiTemporalJoinResult<K, V1, V2>> {
+
+      // State used to hold all right stream values
+      @StateId("rightStream")
+      private final StateSpec<BagState<TimestampedValue<V2>>> rightStream;
+      // State used to hold all left stream values
+      @StateId("leftStream")
+      private final StateSpec<BagState<TimestampedValue<V1>>> leftStream;
+      // The timestamp of the existing processing timer Null if not set
+      @StateId("timerTimestamp")
+      private final StateSpec<ValueState<Long>> timerTimestamp =
+          StateSpecs.value(VarLongCoder.of());
+      // The key, which is required to create a unique window-key for 
optimization cache
+      @StateId("dataKey")
+      private final StateSpec<ValueState<K>> dataKey;
+      // The process timer used to match left stream values to right stream 
values
+      @TimerId("processTimer")
+      private final TimerSpec processTimer = 
TimerSpecs.timer(TimeDomain.EVENT_TIME);
+
+      // ----------- State Variables
+      // Completed work timestamp, used for the GC to indicate the work it 
needs to do.
+      @StateId("lastProcessedLeftStreamTimestampState")
+      private final StateSpec<ValueState<Long>> 
lastProcessedLeftStreamTimestampState =
+          StateSpecs.value(VarLongCoder.of());
+      // Bundle storage of sorted list, used to assist wih O(n^2) problem 
until Sorted Map is
+      // available
+      Map<WindowedValue, List<TimestampedValue<V1>>> leftCache;
+      Map<WindowedValue, List<TimestampedValue<V2>>> rightCache;
+      Coder<V1> leftCoder;
+      Coder<V2> rightCoder;
+      Coder<K> keyCoder;
+
+      private StreamMatcher(Coder<K> keyCoder, Coder<V1> leftCoder, Coder<V2> 
rightCoder) {
+        this.leftCoder = leftCoder;
+        this.rightCoder = rightCoder;
+        this.keyCoder = keyCoder;
+        this.rightStream = 
StateSpecs.bag(TimestampedValue.TimestampedValueCoder.of(rightCoder));
+        this.leftStream = 
StateSpecs.bag(TimestampedValue.TimestampedValueCoder.of(leftCoder));
+        this.dataKey = StateSpecs.value(keyCoder);
+      }
+
+      private static <K, V1, V2> BiTemporalJoinResult<K, V1, V2> createMatch(
+          K key, List<TimestampedValue<V2>> right, TimestampedValue<V1> left, 
int idx) {
+
+        BiTemporalJoinResult<K, V1, V2> m = new BiTemporalJoinResult<>();
+        m.setLeftData(KV.of(key, left.getValue()), left.getTimestamp());
+
+        int adjustedIdx;
+        TimestampedValue<V2> value;
+
+        // If exact match found or if the largest timestamp in the list is 
smaller than the right
+        // streams timestamp id
+        // Note if there are duplicate for the same timestamp the results will 
be non-deterministic
+        if (idx >= 0) {
+          adjustedIdx = idx;
+          value = right.get(adjustedIdx);
+          m.setRightData(KV.of(key, value.getValue()), value.getTimestamp());
+          m.setMatched(true);
+        }
+        if (right.size() > 0 && idx == -1 - right.size()) {
+          adjustedIdx = right.size() - 1;
+          value = right.get(adjustedIdx);
+          m.setRightData(KV.of(key, value.getValue()), value.getTimestamp());
+          m.setMatched(true);
+        }
+        if (idx < -1) {
+          adjustedIdx = Math.abs(idx) - 2;
+          value = right.get(adjustedIdx);
+          m.setRightData(KV.of(key, value.getValue()), value.getTimestamp());
+          m.setMatched(true);
+        }
+        if (idx == -1) {
+          m.setMatched(false);
+        }
+
+        return m;
+      }
+
+      /*
+       Right stream values are added to the bag
+       Left stream values are added to the bag
+       A timer is set if no timer exists already or if min(timerTimestamp) is 
> current object timestamp
+
+       If timertimestamp is NULL then a timer is set in OnProcess()
+       If On OnTimer firing there are elements in the left stream where  
EventTime > OnTimer.timestamp then a new timer is also set.
+      */
+      @ProcessElement
+      public void process(
+          @Element KV<K, BiTemporalJoinResult<K, V1, V2>> input,
+          @Timestamp Instant timestamp,
+          @StateId("rightStream") BagState<TimestampedValue<V2>> rightStream,
+          @StateId("leftStream") BagState<TimestampedValue<V1>> leftStream,
+          @StateId("timerTimestamp") ValueState<Long> timerTimestamp,
+          @StateId("dataKey") ValueState<K> dataKey,
+          @TimerId("processTimer") Timer processTimer) {
+
+        // This workaround is due to OnTimerContext not supporting key access
+        if (dataKey.read() == null) {
+          dataKey.write(input.getKey());
+        }
+
+        if (input.getValue().getRightData() != null) {
+          TimestampedValue<V2> rightData = input.getValue().getRightData();
+          rightStream.add(rightData);
+        }
+
+        if (input.getValue().getLeftData() != null) {
+          TimestampedValue<V1> rightStreamValue = 
input.getValue().getLeftData();
+          leftStream.add(rightStreamValue);
+
+          long processTimerTimestamp = 
Optional.ofNullable(timerTimestamp.read()).orElse(0L);
 
 Review comment:
   Timestamp 0 seems no to be ideal default value, because it can (in theory) 
be valid timestamp that was actually set by an element. Maybe 
`BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis()` would be better. And maybe 
instead of `long` the state could hold the `Instant`? `InstantCoder` should be 
efficient enough.
 
----------------------------------------------------------------
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:
us...@infra.apache.org


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

    Worklog Id:     (was: 294774)
    Time Spent: 6h  (was: 5h 50m)

> Add Utility BiTemporalStreamJoin
> --------------------------------
>
>                 Key: BEAM-7386
>                 URL: https://issues.apache.org/jira/browse/BEAM-7386
>             Project: Beam
>          Issue Type: Improvement
>          Components: sdk-ideas
>    Affects Versions: 2.12.0
>            Reporter: Reza ardeshir rokni
>            Assignee: Reza ardeshir rokni
>            Priority: Major
>          Time Spent: 6h
>  Remaining Estimate: 0h
>
> Add utility class that enables a temporal join between two streams where 
> Stream A is matched to Stream B where
> A.timestamp = (max(b.timestamp) where b.timestamp <= a.timestamp) 
> This will use the following overall flow:
> KV(key, Timestamped<V>) 
> | Window
> | GBK
> | Statefull DoFn
>  
>  



--
This message was sent by Atlassian JIRA
(v7.6.14#76016)

Reply via email to