You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2022/09/19 08:52:40 UTC

[GitHub] [beam] mosche commented on a diff in pull request #22446: Improved pipeline translation in SparkStructuredStreamingRunner

mosche commented on code in PR #22446:
URL: https://github.com/apache/beam/pull/22446#discussion_r974006143


##########
runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/Aggregators.java:
##########
@@ -0,0 +1,590 @@
+/*
+ * 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.runners.spark.structuredstreaming.translation.batch;
+
+import static org.apache.beam.runners.spark.structuredstreaming.translation.helpers.EncoderHelpers.collectionEncoder;
+import static org.apache.beam.runners.spark.structuredstreaming.translation.helpers.EncoderHelpers.encoderOf;
+import static org.apache.beam.runners.spark.structuredstreaming.translation.helpers.EncoderHelpers.mapEncoder;
+import static org.apache.beam.runners.spark.structuredstreaming.translation.helpers.EncoderHelpers.mutablePairEncoder;
+import static org.apache.beam.sdk.transforms.windowing.PaneInfo.NO_FIRING;
+import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterators.peekingIterator;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.function.BiFunction;
+import java.util.function.BinaryOperator;
+import java.util.function.Function;
+import javax.annotation.Nullable;
+import org.apache.beam.runners.spark.structuredstreaming.translation.utils.ScalaInterop.Fun1;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.Sessions;
+import org.apache.beam.sdk.transforms.windowing.TimestampCombiner;
+import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.WindowingStrategy;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Collections2;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.PeekingIterator;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Sets;
+import org.apache.spark.sql.Encoder;
+import org.apache.spark.sql.expressions.Aggregator;
+import org.apache.spark.util.MutablePair;
+import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
+import org.checkerframework.checker.nullness.qual.NonNull;
+import org.checkerframework.checker.nullness.qual.PolyNull;
+import org.joda.time.Instant;
+
+public class Aggregators {
+
+  /**
+   * Creates simple value {@link Aggregator} that is not window aware.
+   *
+   * @param <ValT> {@link CombineFn} input type
+   * @param <AccT> {@link CombineFn} accumulator type
+   * @param <ResT> {@link CombineFn} / {@link Aggregator} result type
+   * @param <InT> {@link Aggregator} input type
+   */
+  public static <ValT, AccT, ResT, InT> Aggregator<InT, ?, ResT> value(
+      CombineFn<ValT, AccT, ResT> fn,
+      Fun1<InT, ValT> valueFn,
+      Encoder<AccT> accEnc,
+      Encoder<ResT> outEnc) {
+    return new ValueAggregator<>(fn, valueFn, accEnc, outEnc);
+  }
+
+  /**
+   * Creates windowed Spark {@link Aggregator} depending on the provided Beam {@link WindowFn}s.
+   *
+   * <p>Specialised implementations are provided for:
+   * <li>{@link Sessions}
+   * <li>Non merging window functions
+   * <li>Merging window functions
+   *
+   * @param <ValT> {@link CombineFn} input type
+   * @param <AccT> {@link CombineFn} accumulator type
+   * @param <ResT> {@link CombineFn} / {@link Aggregator} result type
+   * @param <InT> {@link Aggregator} input type
+   */
+  public static <ValT, AccT, ResT, InT>
+      Aggregator<WindowedValue<InT>, ?, Collection<WindowedValue<ResT>>> windowedValue(
+          CombineFn<ValT, AccT, ResT> fn,
+          Fun1<WindowedValue<InT>, ValT> valueFn,
+          WindowingStrategy<?, ?> windowing,
+          Encoder<BoundedWindow> windowEnc,
+          Encoder<AccT> accEnc,
+          Encoder<WindowedValue<ResT>> outEnc) {
+    if (!windowing.needsMerge()) {
+      return new NonMergingWindowedAggregator<>(fn, valueFn, windowing, windowEnc, accEnc, outEnc);
+    } else if (windowing.getWindowFn().getClass().equals(Sessions.class)) {
+      return new SessionsAggregator<>(fn, valueFn, windowing, (Encoder) windowEnc, accEnc, outEnc);
+    }
+    return new MergingWindowedAggregator<>(fn, valueFn, windowing, windowEnc, accEnc, outEnc);
+  }
+
+  /**
+   * Simple value {@link Aggregator} that is not window aware.
+   *
+   * @param <ValT> {@link CombineFn} input type
+   * @param <AccT> {@link CombineFn} accumulator type
+   * @param <ResT> {@link CombineFn} / {@link Aggregator} result type
+   * @param <InT> {@link Aggregator} input type
+   */
+  private static class ValueAggregator<ValT, AccT, ResT, InT>
+      extends CombineFnAggregator<ValT, AccT, ResT, InT, AccT, ResT> {
+
+    public ValueAggregator(
+        CombineFn<ValT, AccT, ResT> fn,
+        Fun1<InT, ValT> valueFn,
+        Encoder<AccT> accEnc,
+        Encoder<ResT> outEnc) {
+      super(fn, valueFn, accEnc, outEnc);
+    }
+
+    @Override
+    public AccT zero() {
+      return emptyAcc();
+    }
+
+    @Override
+    public AccT reduce(AccT buff, InT in) {
+      return addToAcc(buff, value(in));
+    }
+
+    @Override
+    public AccT merge(AccT b1, AccT b2) {
+      return mergeAccs(b1, b2);
+    }
+
+    @Override
+    public ResT finish(AccT buff) {
+      return extract(buff);
+    }
+  }
+
+  /**
+   * Specialized windowed Spark {@link Aggregator} for Beam {@link WindowFn}s of type {@link
+   * Sessions}. The aggregator uses a {@link TreeMap} as buffer to maintain ordering of the {@link
+   * IntervalWindow}s and merge these more efficiently.
+   *
+   * <p>For efficiency, this aggregator re-implements {@link
+   * Sessions#mergeWindows(WindowFn.MergeContext)} to leverage the already sorted buffer.
+   *
+   * @param <ValT> {@link CombineFn} input type
+   * @param <AccT> {@link CombineFn} accumulator type
+   * @param <ResT> {@link CombineFn} / {@link Aggregator} result type
+   * @param <InT> {@link Aggregator} input type
+   */
+  private static class SessionsAggregator<ValT, AccT, ResT, InT>
+      extends WindowedAggregator<
+          ValT,
+          AccT,
+          ResT,
+          InT,
+          IntervalWindow,
+          TreeMap<IntervalWindow, MutablePair<Instant, AccT>>> {
+
+    SessionsAggregator(
+        CombineFn<ValT, AccT, ResT> combineFn,
+        Fun1<WindowedValue<InT>, ValT> valueFn,
+        WindowingStrategy<?, ?> windowing,
+        Encoder<IntervalWindow> windowEnc,
+        Encoder<AccT> accEnc,
+        Encoder<WindowedValue<ResT>> outEnc) {
+      super(combineFn, valueFn, windowing, windowEnc, accEnc, outEnc, (Class) TreeMap.class);
+      checkArgument(windowing.getWindowFn().getClass().equals(Sessions.class));
+    }
+
+    @Override
+    public final TreeMap<IntervalWindow, MutablePair<Instant, AccT>> zero() {
+      return new TreeMap<>();
+    }
+
+    @Override
+    @SuppressWarnings("keyfor")
+    public TreeMap<IntervalWindow, MutablePair<Instant, AccT>> reduce(
+        TreeMap<IntervalWindow, MutablePair<Instant, AccT>> buff, WindowedValue<InT> input) {
+      for (IntervalWindow window : (Collection<IntervalWindow>) input.getWindows()) {
+        @MonotonicNonNull MutablePair<Instant, AccT> acc = null;
+        @MonotonicNonNull IntervalWindow first = null, last = null;
+        // start with window before or equal to new window (if exists)
+        @Nullable Entry<IntervalWindow, MutablePair<Instant, AccT>> lower = buff.floorEntry(window);
+        if (lower != null && window.intersects(lower.getKey())) {
+          // if intersecting, init accumulator and extend window to span both
+          acc = lower.getValue();
+          window = window.span(lower.getKey());
+          first = last = lower.getKey();
+        }
+        // merge following windows in order while intersecting
+        for (Entry<IntervalWindow, MutablePair<Instant, AccT>> entry :
+            buff.tailMap(window, false).entrySet()) {
+          MutablePair<Instant, AccT> entryAcc = entry.getValue();
+          IntervalWindow entryWindow = entry.getKey();
+          if (window.intersects(entryWindow)) {
+            // extend window and merge accumulators
+            window = window.span(entryWindow);
+            acc = acc == null ? entryAcc : mergeAccs(window, acc, entryAcc);
+            if (first == null) {
+              first = last = entryWindow;
+            } else {
+              last = entryWindow;
+            }
+          } else {
+            break; // stop, later windows won't intersect either
+          }
+        }
+        if (first != null && last != null) {
+          // remove entire merged subset from first to last
+          buff.navigableKeySet().subSet(first, true, last, true).clear();
+        }
+        // add input and get accumulator for new (potentially merged) window
+        buff.put(window, addToAcc(window, acc, value(input), input.getTimestamp()));
+      }
+      return buff;
+    }
+
+    @Override
+    public TreeMap<IntervalWindow, MutablePair<Instant, AccT>> merge(
+        TreeMap<IntervalWindow, MutablePair<Instant, AccT>> b1,
+        TreeMap<IntervalWindow, MutablePair<Instant, AccT>> b2) {
+      if (b1.isEmpty()) {
+        return b2;
+      } else if (b2.isEmpty()) {
+        return b1;
+      }
+      // Init new tree map to merge both buffers
+      TreeMap<IntervalWindow, MutablePair<Instant, AccT>> res = zero();
+      PeekingIterator<Entry<IntervalWindow, MutablePair<Instant, AccT>>> it1 =
+          peekingIterator(b1.entrySet().iterator());
+      PeekingIterator<Entry<IntervalWindow, MutablePair<Instant, AccT>>> it2 =
+          peekingIterator(b2.entrySet().iterator());
+
+      @Nullable MutablePair<Instant, AccT> acc = null;

Review Comment:
   You're right logically, but checker cannot prove that and will fail ...



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

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org