You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by tg...@apache.org on 2017/03/23 21:22:10 UTC

[1/2] beam git commit: This closes #2284

Repository: beam
Updated Branches:
  refs/heads/master def96a2bb -> 890bc1a23


This closes #2284


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/890bc1a2
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/890bc1a2
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/890bc1a2

Branch: refs/heads/master
Commit: 890bc1a23f493b042f8c2de5c042970ce5ddca96
Parents: def96a2 036b3da
Author: Thomas Groh <tg...@google.com>
Authored: Thu Mar 23 14:21:58 2017 -0700
Committer: Thomas Groh <tg...@google.com>
Committed: Thu Mar 23 14:21:58 2017 -0700

----------------------------------------------------------------------
 .../direct/WindowEvaluatorFactoryTest.java      |  5 +-
 .../apache/beam/sdk/testing/StaticWindows.java  | 23 +++++++--
 .../sdk/transforms/windowing/GlobalWindows.java | 17 ++++++-
 .../transforms/windowing/InvalidWindows.java    |  2 +-
 .../windowing/PartitioningWindowFn.java         | 23 ++++++---
 .../beam/sdk/transforms/windowing/Sessions.java |  2 +-
 .../transforms/windowing/SlidingWindows.java    | 32 +++++++++----
 .../beam/sdk/transforms/windowing/WindowFn.java | 18 +++++--
 .../transforms/windowing/WindowMappingFn.java   | 49 ++++++++++++++++++++
 .../apache/beam/sdk/util/IdentityWindowFn.java  |  3 +-
 .../windowing/CalendarWindowsTest.java          | 33 +++++++++++++
 .../transforms/windowing/FixedWindowsTest.java  | 33 +++++++++++++
 .../windowing/SlidingWindowsTest.java           | 16 +++++--
 .../sdk/transforms/windowing/WindowTest.java    |  2 +-
 .../sdk/util/IdentitySideInputWindowFn.java     | 16 ++++++-
 .../sdk/io/gcp/bigquery/BigQueryIOTest.java     |  3 +-
 16 files changed, 239 insertions(+), 38 deletions(-)
----------------------------------------------------------------------



[2/2] beam git commit: Add WindowMappingFn

Posted by tg...@apache.org.
Add WindowMappingFn

WindowMappingFn maps from a main input window to a side input window,
and provides a maximum amount of lookback that is possible to generate
using the MappingFn. This separates the logic of assigning windows from
the logic of mapping main and side inputs. Additionally, it permits
pipeline runners to garbage collect side inputs when they are no longer
accessible.

This is currently unused.

See BEAM-846 and BEAM-260 for additional context.


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/036b3dac
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/036b3dac
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/036b3dac

Branch: refs/heads/master
Commit: 036b3dac97e7307d7831bffaaf3cbeead37b1942
Parents: def96a2
Author: Thomas Groh <tg...@google.com>
Authored: Tue Mar 21 09:46:04 2017 -0700
Committer: Thomas Groh <tg...@google.com>
Committed: Thu Mar 23 14:21:58 2017 -0700

----------------------------------------------------------------------
 .../direct/WindowEvaluatorFactoryTest.java      |  5 +-
 .../apache/beam/sdk/testing/StaticWindows.java  | 23 +++++++--
 .../sdk/transforms/windowing/GlobalWindows.java | 17 ++++++-
 .../transforms/windowing/InvalidWindows.java    |  2 +-
 .../windowing/PartitioningWindowFn.java         | 23 ++++++---
 .../beam/sdk/transforms/windowing/Sessions.java |  2 +-
 .../transforms/windowing/SlidingWindows.java    | 32 +++++++++----
 .../beam/sdk/transforms/windowing/WindowFn.java | 18 +++++--
 .../transforms/windowing/WindowMappingFn.java   | 49 ++++++++++++++++++++
 .../apache/beam/sdk/util/IdentityWindowFn.java  |  3 +-
 .../windowing/CalendarWindowsTest.java          | 33 +++++++++++++
 .../transforms/windowing/FixedWindowsTest.java  | 33 +++++++++++++
 .../windowing/SlidingWindowsTest.java           | 16 +++++--
 .../sdk/transforms/windowing/WindowTest.java    |  2 +-
 .../sdk/util/IdentitySideInputWindowFn.java     | 16 ++++++-
 .../sdk/io/gcp/bigquery/BigQueryIOTest.java     |  3 +-
 16 files changed, 239 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/036b3dac/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java
index ca52852..a71a75b 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java
@@ -45,6 +45,7 @@ import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
 import org.apache.beam.sdk.transforms.windowing.Window;
 import org.apache.beam.sdk.transforms.windowing.Window.Bound;
 import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.transforms.windowing.WindowMappingFn;
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.values.PCollection;
 import org.hamcrest.Matchers;
@@ -313,8 +314,8 @@ public class WindowEvaluatorFactoryTest {
     }
 
     @Override
-    public BoundedWindow getSideInputWindow(BoundedWindow window) {
-      return null;
+    public WindowMappingFn<BoundedWindow> getDefaultWindowMappingFn() {
+      throw new UnsupportedOperationException("Cannot be used as a side input");
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/beam/blob/036b3dac/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/StaticWindows.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/StaticWindows.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/StaticWindows.java
index 949ecac..4be88c8 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/StaticWindows.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/StaticWindows.java
@@ -28,6 +28,8 @@ import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.transforms.windowing.NonMergingWindowFn;
 import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.transforms.windowing.WindowMappingFn;
+import org.joda.time.Duration;
 
 /**
  * A {@link WindowFn} that assigns all elements to a static collection of
@@ -100,9 +102,22 @@ final class StaticWindows extends NonMergingWindowFn<Object, BoundedWindow> {
   }
 
   @Override
-  public BoundedWindow getSideInputWindow(BoundedWindow window) {
-    checkArgument(windows.get().contains(window),
-        "StaticWindows only supports side input windows for main input windows that it contains");
-    return window;
+  public WindowMappingFn<BoundedWindow> getDefaultWindowMappingFn() {
+    return new WindowMappingFn<BoundedWindow>() {
+      @Override
+      public BoundedWindow getSideInputWindow(BoundedWindow mainWindow) {
+        checkArgument(
+            windows.get().contains(mainWindow),
+            "%s only supports side input windows for main input windows that it contains",
+            StaticWindows.class.getSimpleName());
+        return mainWindow;
+      }
+
+      @Override
+      public Duration maximumLookback() {
+        // TODO: This may be unsafe.
+        return Duration.millis(Long.MAX_VALUE);
+      }
+    };
   }
 }

http://git-wip-us.apache.org/repos/asf/beam/blob/036b3dac/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindows.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindows.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindows.java
index 6606a5a..e91fad1 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindows.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindows.java
@@ -20,6 +20,7 @@ package org.apache.beam.sdk.transforms.windowing;
 import java.util.Collection;
 import java.util.Collections;
 import org.apache.beam.sdk.coders.Coder;
+import org.joda.time.Duration;
 import org.joda.time.Instant;
 
 /**
@@ -46,8 +47,20 @@ public class GlobalWindows extends NonMergingWindowFn<Object, GlobalWindow> {
   }
 
   @Override
-  public GlobalWindow getSideInputWindow(BoundedWindow window) {
-    return GlobalWindow.INSTANCE;
+  public WindowMappingFn<GlobalWindow> getDefaultWindowMappingFn() {
+    return new GlobalWindowMappingFn();
+  }
+
+  static class GlobalWindowMappingFn extends WindowMappingFn<GlobalWindow> {
+    @Override
+    public GlobalWindow getSideInputWindow(BoundedWindow mainWindow) {
+      return GlobalWindow.INSTANCE;
+    }
+
+    @Override
+    public Duration maximumLookback() {
+      return Duration.ZERO;
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/beam/blob/036b3dac/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/InvalidWindows.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/InvalidWindows.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/InvalidWindows.java
index cbbc937..92041fc 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/InvalidWindows.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/InvalidWindows.java
@@ -75,7 +75,7 @@ public class InvalidWindows<W extends BoundedWindow> extends WindowFn<Object, W>
   }
 
   @Override
-  public W getSideInputWindow(BoundedWindow window) {
+  public WindowMappingFn<W> getDefaultWindowMappingFn() {
     throw new UnsupportedOperationException("InvalidWindows is not allowed in side inputs");
   }
 

http://git-wip-us.apache.org/repos/asf/beam/blob/036b3dac/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PartitioningWindowFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PartitioningWindowFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PartitioningWindowFn.java
index b84f891..40cff8a 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PartitioningWindowFn.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PartitioningWindowFn.java
@@ -19,6 +19,7 @@ package org.apache.beam.sdk.transforms.windowing;
 
 import java.util.Arrays;
 import java.util.Collection;
+import org.joda.time.Duration;
 import org.joda.time.Instant;
 
 /**
@@ -41,12 +42,22 @@ public abstract class PartitioningWindowFn<T, W extends BoundedWindow>
   }
 
   @Override
-  public W getSideInputWindow(final BoundedWindow window) {
-    if (window instanceof GlobalWindow) {
-      throw new IllegalArgumentException(
-          "Attempted to get side input window for GlobalWindow from non-global WindowFn");
-    }
-    return assignWindow(window.maxTimestamp());
+  public WindowMappingFn<W> getDefaultWindowMappingFn() {
+    return new WindowMappingFn<W>() {
+      @Override
+      public W getSideInputWindow(BoundedWindow mainWindow) {
+        if (mainWindow instanceof GlobalWindow) {
+          throw new IllegalArgumentException(
+              "Attempted to get side input window for GlobalWindow from non-global WindowFn");
+        }
+        return assignWindow(mainWindow.maxTimestamp());
+      }
+
+      @Override
+      public Duration maximumLookback() {
+        return Duration.ZERO;
+      }
+    };
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/beam/blob/036b3dac/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Sessions.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Sessions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Sessions.java
index 14d68f9..5cc7c65 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Sessions.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Sessions.java
@@ -80,7 +80,7 @@ public class Sessions extends WindowFn<Object, IntervalWindow> {
   }
 
   @Override
-  public IntervalWindow getSideInputWindow(BoundedWindow window) {
+  public WindowMappingFn<IntervalWindow> getDefaultWindowMappingFn() {
     throw new UnsupportedOperationException("Sessions is not allowed in side inputs");
   }
 

http://git-wip-us.apache.org/repos/asf/beam/blob/036b3dac/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/SlidingWindows.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/SlidingWindows.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/SlidingWindows.java
index 1eb56f7..b27f4e6 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/SlidingWindows.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/SlidingWindows.java
@@ -108,9 +108,12 @@ public class SlidingWindows extends NonMergingWindowFn<Object, IntervalWindow> {
 
   @Override
   public Collection<IntervalWindow> assignWindows(AssignContext c) {
+    return assignWindows(c.timestamp());
+  }
+
+  public Collection<IntervalWindow> assignWindows(Instant timestamp) {
     List<IntervalWindow> windows =
         new ArrayList<>((int) (size.getMillis() / period.getMillis()));
-    Instant timestamp = c.timestamp();
     long lastStart = lastStartFor(timestamp);
     for (long start = lastStart;
          start > timestamp.minus(size).getMillis();
@@ -121,16 +124,27 @@ public class SlidingWindows extends NonMergingWindowFn<Object, IntervalWindow> {
   }
 
   /**
-   * Return the earliest window that contains the end of the main-input window.
+   * Return a {@link WindowMappingFn} that returns the earliest window that contains the end of the
+   * main-input window.
    */
   @Override
-  public IntervalWindow getSideInputWindow(final BoundedWindow window) {
-    if (window instanceof GlobalWindow) {
-      throw new IllegalArgumentException(
-          "Attempted to get side input window for GlobalWindow from non-global WindowFn");
-    }
-    long lastStart = lastStartFor(window.maxTimestamp().minus(size));
-    return new IntervalWindow(new Instant(lastStart + period.getMillis()), size);
+  public WindowMappingFn<IntervalWindow> getDefaultWindowMappingFn() {
+    return new WindowMappingFn<IntervalWindow>() {
+      @Override
+      public IntervalWindow getSideInputWindow(BoundedWindow mainWindow) {
+        if (mainWindow instanceof GlobalWindow) {
+          throw new IllegalArgumentException(
+              "Attempted to get side input window for GlobalWindow from non-global WindowFn");
+        }
+        long lastStart = lastStartFor(mainWindow.maxTimestamp().minus(size));
+        return new IntervalWindow(new Instant(lastStart + period.getMillis()), size);
+      }
+
+      @Override
+      public Duration maximumLookback() {
+        return Duration.ZERO;
+      }
+    };
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/beam/blob/036b3dac/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/WindowFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/WindowFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/WindowFn.java
index ea0bb79..2f9e6c1 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/WindowFn.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/WindowFn.java
@@ -125,11 +125,23 @@ public abstract class WindowFn<T, W extends BoundedWindow>
 
   /**
    * Returns the window of the side input corresponding to the given window of
-   * the main input.
+   * the main input. If not overridden, will use the window returned by calling
+   * {@link WindowMappingFn#getSideInputWindow(BoundedWindow)} on the result of
+   * {@link #getDefaultWindowMappingFn()}.
    *
-   * <p>Authors of custom {@code WindowFn}s should override this.
+   * @deprecated see {@link #getDefaultWindowMappingFn()}
    */
-  public abstract W getSideInputWindow(BoundedWindow window);
+  @Deprecated
+  public W getSideInputWindow(BoundedWindow window) {
+    return getDefaultWindowMappingFn().getSideInputWindow(window);
+  }
+
+  /**
+   * Returns the default {@link WindowMappingFn} to use to map main input windows to side input
+   * windows. This should accept arbitrary main input windows, and produce a {@link BoundedWindow}
+   * that can be produced by this {@link WindowFn}.
+   */
+  public abstract WindowMappingFn<W> getDefaultWindowMappingFn();
 
   /**
    * Returns the output timestamp to use for data depending on the given

http://git-wip-us.apache.org/repos/asf/beam/blob/036b3dac/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/WindowMappingFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/WindowMappingFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/WindowMappingFn.java
new file mode 100644
index 0000000..62bf544
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/WindowMappingFn.java
@@ -0,0 +1,49 @@
+/*
+ * 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.windowing;
+
+import java.io.Serializable;
+import org.apache.beam.sdk.transforms.ParDo.BoundMulti;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.joda.time.Duration;
+
+/**
+ * A function that takes the windows of elements in a main input and maps them to the appropriate
+ * window in a {@link PCollectionView} consumed as a
+ * {@link BoundMulti#withSideInputs(PCollectionView[]) side input}.
+ */
+public abstract class WindowMappingFn<TargetWindowT extends BoundedWindow> implements Serializable {
+  /**
+   * Returns the window of the side input corresponding to the given window of the main input.
+   */
+  public abstract TargetWindowT getSideInputWindow(BoundedWindow mainWindow);
+
+  /**
+   * The maximum distance between the end of any main input window {@code mainWindow}
+   * and the end of the side input window returned by {@link #getSideInputWindow(BoundedWindow)}
+   *
+   * <p>A side input window {@code w} becomes unreachable when the input watermarks for all
+   * consumers surpasses the timestamp:
+   *
+   * <p>(end of side input window) + (maximum lookback) + (main input allowed lateness).
+   *
+   * <p>At this point, every main input window that could map to {@code w} is expired.
+   */
+  public abstract Duration maximumLookback();
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/036b3dac/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IdentityWindowFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IdentityWindowFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IdentityWindowFn.java
index c02e1f4..d5f2181 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IdentityWindowFn.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IdentityWindowFn.java
@@ -27,6 +27,7 @@ import org.apache.beam.sdk.transforms.windowing.InvalidWindows;
 import org.apache.beam.sdk.transforms.windowing.NonMergingWindowFn;
 import org.apache.beam.sdk.transforms.windowing.Window;
 import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.transforms.windowing.WindowMappingFn;
 import org.apache.beam.sdk.values.PCollection;
 import org.joda.time.Instant;
 
@@ -89,7 +90,7 @@ public class IdentityWindowFn<T> extends NonMergingWindowFn<T, BoundedWindow> {
   }
 
   @Override
-  public BoundedWindow getSideInputWindow(BoundedWindow window) {
+  public WindowMappingFn<BoundedWindow> getDefaultWindowMappingFn() {
     throw new UnsupportedOperationException(
         String.format(
             "%s.getSideInputWindow() should never be called."

http://git-wip-us.apache.org/repos/asf/beam/blob/036b3dac/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/CalendarWindowsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/CalendarWindowsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/CalendarWindowsTest.java
index 54cdd06..cd562e9 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/CalendarWindowsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/CalendarWindowsTest.java
@@ -21,6 +21,7 @@ import static org.apache.beam.sdk.testing.WindowFnTestUtils.runWindowFn;
 import static org.apache.beam.sdk.testing.WindowFnTestUtils.set;
 import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem;
 import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.equalTo;
 import static org.junit.Assert.assertEquals;
 
 import java.util.Arrays;
@@ -29,11 +30,16 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.transforms.windowing.CalendarWindows.MonthsWindows;
+import org.hamcrest.Matchers;
 import org.joda.time.DateTime;
 import org.joda.time.DateTimeConstants;
 import org.joda.time.DateTimeZone;
+import org.joda.time.Duration;
 import org.joda.time.Instant;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.ExpectedException;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
@@ -42,6 +48,7 @@ import org.junit.runners.JUnit4;
  */
 @RunWith(JUnit4.class)
 public class CalendarWindowsTest {
+  @Rule public ExpectedException thrown = ExpectedException.none();
 
   private static Instant makeTimestamp(int year, int month, int day, int hours, int minutes) {
     return new DateTime(year, month, day, hours, minutes, DateTimeZone.UTC).toInstant();
@@ -262,6 +269,32 @@ public class CalendarWindowsTest {
   }
 
   @Test
+  public void testDefaultWindowMappingFn() {
+    MonthsWindows windowFn = CalendarWindows.months(2);
+    WindowMappingFn<?> mapping = windowFn.getDefaultWindowMappingFn();
+
+    assertThat(
+        mapping.getSideInputWindow(
+            new BoundedWindow() {
+              @Override
+              public Instant maxTimestamp() {
+                return new Instant(100L);
+              }
+            }),
+        Matchers.<BoundedWindow>equalTo(windowFn.assignWindow(new Instant(100L))));
+    assertThat(mapping.maximumLookback(), equalTo(Duration.ZERO));
+  }
+
+  @Test
+  public void testDefaultWindowMappingFnGlobal() {
+    MonthsWindows windowFn = CalendarWindows.months(2);
+    WindowMappingFn<?> mapping = windowFn.getDefaultWindowMappingFn();
+
+    thrown.expect(IllegalArgumentException.class);
+    mapping.getSideInputWindow(GlobalWindow.INSTANCE);
+  }
+
+  @Test
   public void testDisplayData() {
     DateTimeZone timeZone = DateTimeZone.forID("America/Los_Angeles");
     Instant jan1 = new DateTime(1990, 1, 1, 0, 0, timeZone).toInstant();

http://git-wip-us.apache.org/repos/asf/beam/blob/036b3dac/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/FixedWindowsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/FixedWindowsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/FixedWindowsTest.java
index afa5d60..47c273a 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/FixedWindowsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/FixedWindowsTest.java
@@ -21,6 +21,7 @@ import static org.apache.beam.sdk.testing.WindowFnTestUtils.runWindowFn;
 import static org.apache.beam.sdk.testing.WindowFnTestUtils.set;
 import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem;
 import static org.hamcrest.CoreMatchers.containsString;
+import static org.hamcrest.Matchers.equalTo;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertThat;
@@ -33,9 +34,12 @@ import java.util.Map;
 import java.util.Set;
 import org.apache.beam.sdk.testing.WindowFnTestUtils;
 import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.hamcrest.Matchers;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.ExpectedException;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
@@ -44,6 +48,7 @@ import org.junit.runners.JUnit4;
  */
 @RunWith(JUnit4.class)
 public class FixedWindowsTest {
+  @Rule public ExpectedException thrown = ExpectedException.none();
 
   @Test
   public void testSimpleFixedWindow() throws Exception {
@@ -83,6 +88,34 @@ public class FixedWindowsTest {
             Arrays.asList(1L, 2L, 1000L, 5000L, 5001L, 10000L)));
   }
 
+  @Test
+  public void testDefaultWindowMappingFn() {
+    PartitioningWindowFn<?, ?> windowFn = FixedWindows.of(Duration.standardMinutes(20L));
+    WindowMappingFn<?> mapping = windowFn.getDefaultWindowMappingFn();
+
+    assertThat(
+        mapping.getSideInputWindow(
+            new BoundedWindow() {
+              @Override
+              public Instant maxTimestamp() {
+                return new Instant(100L);
+              }
+            }),
+        Matchers.<BoundedWindow>equalTo(
+            new IntervalWindow(
+                new Instant(0L), new Instant(0L).plus(Duration.standardMinutes(20L)))));
+    assertThat(mapping.maximumLookback(), equalTo(Duration.ZERO));
+  }
+
+  @Test
+  public void testDefaultWindowMappingFnGlobalWindow() {
+    PartitioningWindowFn<?, ?> windowFn = FixedWindows.of(Duration.standardMinutes(20L));
+    WindowMappingFn<?> mapping = windowFn.getDefaultWindowMappingFn();
+
+    thrown.expect(IllegalArgumentException.class);
+    mapping.getSideInputWindow(GlobalWindow.INSTANCE);
+  }
+
   void checkConstructionFailure(int size, int offset) {
     try {
       FixedWindows.of(Duration.standardSeconds(size)).withOffset(Duration.standardSeconds(offset));

http://git-wip-us.apache.org/repos/asf/beam/blob/036b3dac/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SlidingWindowsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SlidingWindowsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SlidingWindowsTest.java
index 54c01a8..dd673d3 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SlidingWindowsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SlidingWindowsTest.java
@@ -20,6 +20,7 @@ package org.apache.beam.sdk.transforms.windowing;
 import static org.apache.beam.sdk.testing.WindowFnTestUtils.runWindowFn;
 import static org.apache.beam.sdk.testing.WindowFnTestUtils.set;
 import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem;
+import static org.hamcrest.Matchers.equalTo;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertThat;
@@ -152,28 +153,33 @@ public class SlidingWindowsTest {
   }
 
   @Test
-  public void testGetSideInputWindow() {
+  public void testDefaultWindowMappingFn() {
     // [40, 1040), [340, 1340), [640, 1640) ...
     SlidingWindows slidingWindows = SlidingWindows.of(new Duration(1000))
         .every(new Duration(300)).withOffset(new Duration(40));
+    WindowMappingFn<?> mapping =
+        slidingWindows.getDefaultWindowMappingFn();
+
+    assertThat(mapping.maximumLookback(), equalTo(Duration.ZERO));
+
     // Prior
     assertEquals(
         new IntervalWindow(new Instant(340), new Instant(1340)),
-        slidingWindows.getSideInputWindow(
+        mapping.getSideInputWindow(
             new IntervalWindow(new Instant(0), new Instant(1041))));
     assertEquals(
         new IntervalWindow(new Instant(340), new Instant(1340)),
-        slidingWindows.getSideInputWindow(
+        mapping.getSideInputWindow(
             new IntervalWindow(new Instant(0), new Instant(1339))));
     // Align
     assertEquals(
         new IntervalWindow(new Instant(340), new Instant(1340)),
-        slidingWindows.getSideInputWindow(
+        mapping.getSideInputWindow(
             new IntervalWindow(new Instant(0), new Instant(1340))));
     // After
     assertEquals(
         new IntervalWindow(new Instant(640), new Instant(1640)),
-        slidingWindows.getSideInputWindow(
+        mapping.getSideInputWindow(
             new IntervalWindow(new Instant(0), new Instant(1341))));
   }
 

http://git-wip-us.apache.org/repos/asf/beam/blob/036b3dac/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java
index 70635e9..327b7a8 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java
@@ -305,7 +305,7 @@ public class WindowTest implements Serializable {
     }
 
     @Override
-    public IntervalWindow getSideInputWindow(BoundedWindow window) {
+    public WindowMappingFn<IntervalWindow> getDefaultWindowMappingFn() {
       throw new UnsupportedOperationException(
           String.format("Can't use %s for side inputs", getClass().getSimpleName()));
     }

http://git-wip-us.apache.org/repos/asf/beam/blob/036b3dac/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IdentitySideInputWindowFn.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IdentitySideInputWindowFn.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IdentitySideInputWindowFn.java
index 2b48b9f..60d9afe 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IdentitySideInputWindowFn.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IdentitySideInputWindowFn.java
@@ -24,6 +24,8 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
 import org.apache.beam.sdk.transforms.windowing.NonMergingWindowFn;
 import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.transforms.windowing.WindowMappingFn;
+import org.joda.time.Duration;
 
 /**
  * A {@link WindowFn} for use during tests that returns the input window for calls to
@@ -48,7 +50,17 @@ public class IdentitySideInputWindowFn extends NonMergingWindowFn<Integer, Bound
   }
 
   @Override
-  public BoundedWindow getSideInputWindow(BoundedWindow window) {
-    return window;
+  public WindowMappingFn<BoundedWindow> getDefaultWindowMappingFn() {
+    return new WindowMappingFn<BoundedWindow>() {
+      @Override
+      public BoundedWindow getSideInputWindow(BoundedWindow window) {
+        return window;
+      }
+
+      @Override
+      public Duration maximumLookback() {
+        return Duration.ZERO;
+      }
+    };
   }
 }

http://git-wip-us.apache.org/repos/asf/beam/blob/036b3dac/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java
index dcc3800..4dcc8d5 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java
@@ -147,6 +147,7 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
 import org.apache.beam.sdk.transforms.windowing.NonMergingWindowFn;
 import org.apache.beam.sdk.transforms.windowing.Window;
 import org.apache.beam.sdk.transforms.windowing.WindowFn;
+import org.apache.beam.sdk.transforms.windowing.WindowMappingFn;
 import org.apache.beam.sdk.util.CoderUtils;
 import org.apache.beam.sdk.util.IOChannelFactory;
 import org.apache.beam.sdk.util.IOChannelUtils;
@@ -1048,7 +1049,7 @@ public class BigQueryIOTest implements Serializable {
     }
 
     @Override
-    public PartitionedGlobalWindow getSideInputWindow(BoundedWindow window) {
+    public WindowMappingFn<PartitionedGlobalWindow> getDefaultWindowMappingFn() {
       throw new UnsupportedOperationException(
           "PartitionedGlobalWindows is not allowed in side inputs");
     }