Skip to content

Commit

Permalink
This closes apache#2284
Browse files Browse the repository at this point in the history
  • Loading branch information
tgroh committed Mar 23, 2017
2 parents def96a2 + 036b3da commit 890bc1a
Show file tree
Hide file tree
Showing 16 changed files with 239 additions and 38 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,7 @@
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;
Expand Down Expand Up @@ -313,8 +314,8 @@ public Coder<BoundedWindow> windowCoder() {
}

@Override
public BoundedWindow getSideInputWindow(BoundedWindow window) {
return null;
public WindowMappingFn<BoundedWindow> getDefaultWindowMappingFn() {
throw new UnsupportedOperationException("Cannot be used as a side input");
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,8 @@
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
Expand Down Expand Up @@ -100,9 +102,22 @@ public Coder<BoundedWindow> windowCoder() {
}

@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);
}
};
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
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;

/**
Expand All @@ -46,8 +47,20 @@ public Coder<GlobalWindow> windowCoder() {
}

@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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -75,7 +75,7 @@ && getOriginalWindowFn().isCompatible(
}

@Override
public W getSideInputWindow(BoundedWindow window) {
public WindowMappingFn<W> getDefaultWindowMappingFn() {
throw new UnsupportedOperationException("InvalidWindows is not allowed in side inputs");
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@

import java.util.Arrays;
import java.util.Collection;
import org.joda.time.Duration;
import org.joda.time.Instant;

/**
Expand All @@ -41,12 +42,22 @@ public final Collection<W> assignWindows(AssignContext c) {
}

@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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -80,7 +80,7 @@ public boolean isCompatible(WindowFn<?, ?> other) {
}

@Override
public IntervalWindow getSideInputWindow(BoundedWindow window) {
public WindowMappingFn<IntervalWindow> getDefaultWindowMappingFn() {
throw new UnsupportedOperationException("Sessions is not allowed in side inputs");
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -108,9 +108,12 @@ public Coder<IntervalWindow> windowCoder() {

@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();
Expand All @@ -121,16 +124,27 @@ public Collection<IntervalWindow> assignWindows(AssignContext c) {
}

/**
* 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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -125,11 +125,23 @@ public abstract void merge(Collection<W> toBeMerged, W mergeResult)

/**
* 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
Expand Down
Original file line number Diff line number Diff line change
@@ -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:https://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();
}
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
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;

Expand Down Expand Up @@ -89,7 +90,7 @@ public Coder<BoundedWindow> windowCoder() {
}

@Override
public BoundedWindow getSideInputWindow(BoundedWindow window) {
public WindowMappingFn<BoundedWindow> getDefaultWindowMappingFn() {
throw new UnsupportedOperationException(
String.format(
"%s.getSideInputWindow() should never be called."
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
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;
Expand All @@ -29,11 +30,16 @@
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;

Expand All @@ -42,6 +48,7 @@
*/
@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();
Expand Down Expand Up @@ -261,6 +268,32 @@ public void testTimeZone() throws Exception {
timestamps));
}

@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");
Expand Down
Loading

0 comments on commit 890bc1a

Please sign in to comment.