tysonjh commented on a change in pull request #12915:
URL: https://github.com/apache/beam/pull/12915#discussion_r497696164



##########
File path: 
sdks/java/extensions/join-library/src/main/java/org/apache/beam/sdk/extensions/joinlibrary/Join.java
##########
@@ -350,6 +368,276 @@ public void processElement(ProcessContext c) {
     return leftCollection.apply(name, InnerJoin.with(rightCollection));
   }
 
+  /**
+   * PTransform representing a temporal inner join of PCollection<KV>s.
+   *
+   * @param <K> Type of the key for both collections.
+   * @param <V1> Type of the values for the left collection.
+   * @param <V2> Type of the values for the right collection.
+   */
+  public static class TemporalInnerJoin<K, V1, V2>
+      extends PTransform<PCollection<KV<K, V1>>, PCollection<KV<K, KV<V1, 
V2>>>> {
+    private final transient PCollection<KV<K, V2>> rightCollection;
+    private final Duration temporalBound;
+    private final SimpleFunction<KV<V1, V2>, Boolean> comparatorFn;
+
+    private TemporalInnerJoin(
+        final PCollection<KV<K, V2>> rightCollection,
+        final Duration temporalBound,
+        final SimpleFunction<KV<V1, V2>, Boolean> compareFn) {
+      this.temporalBound = temporalBound;
+      this.rightCollection = rightCollection;
+      this.comparatorFn = compareFn;
+    }
+
+    /**
+     * Returns a TemporalInnerJoin PTransform that joins two PCollection<KV>s.
+     *
+     * <p>Similar to {@code innerJoin} but also supports unbounded 
PCollections in the GlobalWindow.
+     * Join results will be produced eagerly as new elements are received, 
regardless of windowing,
+     * however users should prefer {@code innerJoin} in most cases for better 
throughput.
+     *
+     * <p>The non-inclusive {@code temporalBound}, used as part of the join 
predicate, allows
+     * elements to be expired when they are irrelevant according to the 
event-time watermark. This
+     * helps reduce the search space, storage, and memory requirements.
+     *
+     * @param rightCollection Right side collection of the join.
+     * @param temporalBound Duration used in the join predicate 
(non-inclusive).
+     * @param compareFn Join predicate used for matching elements.
+     * @param <K> Type of the key for both collections.
+     * @param <V1> Type of the values for the left collection.
+     * @param <V2> Type of values for the right collection.
+     */
+    public static <K, V1, V2> TemporalInnerJoin<K, V1, V2> with(
+        PCollection<KV<K, V2>> rightCollection,
+        Duration temporalBound,
+        SimpleFunction<KV<V1, V2>, Boolean> compareFn) {
+      return new TemporalInnerJoin<>(rightCollection, temporalBound, 
compareFn);
+    }
+
+    @Override
+    public PCollection<KV<K, KV<V1, V2>>> expand(PCollection<KV<K, V1>> 
leftCollection) {
+      // left        right
+      // tag-left    tag-right (create union type)
+      //   \         /
+      //     flatten
+      //     join
+
+      Coder<K> keyCoder = ((KvCoder<K, V1>) 
leftCollection.getCoder()).getKeyCoder();
+      Coder<V1> leftValueCoder = ((KvCoder<K, V1>) 
leftCollection.getCoder()).getValueCoder();
+      Coder<V2> rightValueCoder = ((KvCoder<K, V2>) 
rightCollection.getCoder()).getValueCoder();
+
+      PCollection<KV<K, KV<V1, V2>>> leftUnion =
+          leftCollection
+              .apply("LeftUnionTag", MapElements.via(new LeftUnionTagFn<K, V1, 
V2>()))
+              .setCoder(
+                  KvCoder.of(
+                      keyCoder,
+                      KvCoder.of(
+                          NullableCoder.of(leftValueCoder), 
NullableCoder.of(rightValueCoder))));
+
+      PCollection<KV<K, KV<V1, V2>>> rightUnion =
+          rightCollection
+              .apply("RightUnionTag", MapElements.via(new RightUnionTagFn<K, 
V1, V2>()))
+              .setCoder(
+                  KvCoder.of(
+                      keyCoder,
+                      KvCoder.of(
+                          NullableCoder.of(leftValueCoder), 
NullableCoder.of(rightValueCoder))));
+
+      return PCollectionList.of(leftUnion)
+          .and(rightUnion)
+          .apply(Flatten.pCollections())
+          .apply(
+              "TemporalInnerJoinFn",
+              ParDo.of(
+                  new TemporalInnerJoinFn<>(
+                      leftValueCoder, rightValueCoder, temporalBound, 
comparatorFn)));
+    }
+  }
+
+  private static class LeftUnionTagFn<K, V1, V2>
+      extends SimpleFunction<KV<K, V1>, KV<K, KV<V1, V2>>> {
+    @Override
+    public KV<K, KV<V1, V2>> apply(KV<K, V1> element) {
+      return KV.of(element.getKey(), KV.of(element.getValue(), null));
+    }
+  }
+
+  private static class RightUnionTagFn<K, V1, V2>
+      extends SimpleFunction<KV<K, V2>, KV<K, KV<V1, V2>>> {
+    @Override
+    public KV<K, KV<V1, V2>> apply(KV<K, V2> element) {
+      return KV.of(element.getKey(), KV.of(null, element.getValue()));
+    }
+  }
+
+  private static class TemporalInnerJoinFn<K, V1, V2>
+      extends DoFn<KV<K, KV<V1, V2>>, KV<K, KV<V1, V2>>> {
+
+    @StateId("left")
+    private final StateSpec<OrderedListState<V1>> leftStateSpec;
+
+    @StateId("right")
+    private final StateSpec<OrderedListState<V2>> rightStateSpec;
+
+    @TimerId("eviction")
+    private final TimerSpec evictionSpec = 
TimerSpecs.timer(TimeDomain.EVENT_TIME);
+
+    private final Duration temporalBound;
+    private final Duration evictionFrequency;
+    private final SimpleFunction<KV<V1, V2>, Boolean> compareFn;
+    private transient boolean evictionTimerInitialized;
+
+    @Setup
+    public void setup() {
+      evictionTimerInitialized = false;
+    }
+
+    protected TemporalInnerJoinFn(
+        final Coder<V1> leftCoder,
+        final Coder<V2> rightCoder,
+        final Duration temporalBound,
+        SimpleFunction<KV<V1, V2>, Boolean> compareFn) {
+      this.leftStateSpec = StateSpecs.orderedList(leftCoder);
+      this.rightStateSpec = StateSpecs.orderedList(rightCoder);
+      this.temporalBound = temporalBound;
+      this.compareFn = compareFn;
+      this.evictionFrequency =
+          temporalBound.getMillis() <= 4 ? Duration.millis(1) : 
temporalBound.dividedBy(4);
+    }
+
+    /**
+     * Finds an element in {@code search} state that satisfies {@code func} 
within the non-inclusive
+     * time range (timestamp - temporalBound, timestamp + temporalBound).
+     *
+     * @param timestamp Instant that scopes the match within temporalBound 
(i.e. midpoint).
+     * @param search State to search.
+     * @param func Join predicate.
+     * @param <T> Type of the state collection and return value.
+     * @return Matching element or null if none is found.
+     */
+    @Nullable
+    private <T> TimestampedValue<T> findMatch(
+        Instant timestamp,
+        OrderedListState<T> search,
+        Function<TimestampedValue<T>, Boolean> func) {
+      Iterable<TimestampedValue<T>> searchIterable =
+          search.readRange(timestamp.minus(temporalBound), 
timestamp.plus(temporalBound));
+      if (searchIterable != null) {
+        for (TimestampedValue<T> current : searchIterable) {
+          if (new Duration(current.getTimestamp(), 
timestamp).abs().isShorterThan(temporalBound)
+              && func.apply(current)) {
+            return current;
+          }
+        }
+      }
+      return null;
+    }
+
+    /**
+     * Removes an element from state. This method is the semantic equivalent 
of an
+     * OrderedListState#remove which does not exist.
+     */
+    private <T> void remove(OrderedListState<T> state, TimestampedValue<T> 
element) {
+      Instant upperBound = element.getTimestamp().plus(1);

Review comment:
       Done. Closing this.




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


Reply via email to