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

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_r313897433
 
 

 ##########
 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;
 
 Review comment:
   Hm, I'm wondering, does this have to work in FixedWindows? I think there 
should be no blocker for this to work on GlobalWindows as well. Even for any 
other windows, provided the windowFn is the same on both sides, right?
 
----------------------------------------------------------------
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: 294777)
    Time Spent: 6.5h  (was: 6h 20m)

> 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: 6.5h
>  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