Skip to content

Commit

Permalink
This closes #1353
Browse files Browse the repository at this point in the history
  • Loading branch information
kennknowles committed Nov 18, 2016
2 parents 212fec4 + c0623c1 commit 4852d2e
Show file tree
Hide file tree
Showing 27 changed files with 579 additions and 482 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -354,13 +354,26 @@ public StateInternals<K> stateInternals() {
}

@Override
public void outputWindowedValue(KV<K, Iterable<V>> output, Instant timestamp,
Collection<? extends BoundedWindow> windows, PaneInfo pane) {
public void outputWindowedValue(
KV<K, Iterable<V>> output,
Instant timestamp,
Collection<? extends BoundedWindow> windows,
PaneInfo pane) {
if (traceTuples) {
LOG.debug("\nemitting {} timestamp {}\n", output, timestamp);
}
ApexGroupByKeyOperator.this.output.emit(ApexStreamTuple.DataTuple.of(
WindowedValue.of(output, timestamp, windows, pane)));
ApexGroupByKeyOperator.this.output.emit(
ApexStreamTuple.DataTuple.of(WindowedValue.of(output, timestamp, windows, pane)));
}

@Override
public <SideOutputT> void sideOutputWindowedValue(
TupleTag<SideOutputT> tag,
SideOutputT output,
Instant timestamp,
Collection<? extends BoundedWindow> windows,
PaneInfo pane) {
throw new UnsupportedOperationException("GroupAlsoByWindow should not use side outputs");
}

@Override
Expand All @@ -379,8 +392,9 @@ public PaneInfo pane() {
}

@Override
public <T> void writePCollectionViewData(TupleTag<?> tag, Iterable<WindowedValue<T>> data,
Coder<T> elemCoder) throws IOException {
public <T> void writePCollectionViewData(
TupleTag<?> tag, Iterable<WindowedValue<T>> data, Coder<T> elemCoder)
throws IOException {
throw new RuntimeException("writePCollectionViewData() not available in Streaming mode.");
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -87,7 +87,8 @@ public void processElement(ProcessContext c) throws Exception {
TriggerStateMachines.stateMachineForTrigger(windowingStrategy.getTrigger())),
stateInternals,
timerInternals,
c.windowingInternals(),
WindowingInternalsAdapters.outputWindowedValue(c.windowingInternals()),
WindowingInternalsAdapters.sideInputReader(c.windowingInternals()),
droppedDueToClosedWindow,
reduceFn,
c.getPipelineOptions());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -69,14 +69,15 @@ public void processElement(
StateInternals<K> stateInternals = stateInternalsFactory.stateInternalsForKey(key);

ReduceFnRunner<K, InputT, OutputT, W> reduceFnRunner =
new ReduceFnRunner<K, InputT, OutputT, W>(
new ReduceFnRunner<>(
key,
strategy,
ExecutableTriggerStateMachine.create(
TriggerStateMachines.stateMachineForTrigger(strategy.getTrigger())),
stateInternals,
timerInternals,
c.windowingInternals(),
WindowingInternalsAdapters.outputWindowedValue(c.windowingInternals()),
WindowingInternalsAdapters.sideInputReader(c.windowingInternals()),
droppedDueToClosedWindow,
reduceFn,
c.getPipelineOptions());
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,45 @@
/*
* 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.core;

import java.util.Collection;
import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
import org.apache.beam.sdk.transforms.windowing.PaneInfo;
import org.apache.beam.sdk.values.TupleTag;
import org.joda.time.Instant;

/**
* An object that can output a value with all of its windowing information to the main output or
* a side output.
*/
public interface OutputWindowedValue<OutputT> {
/** Outputs a value with windowing information to the main output. */
void outputWindowedValue(
OutputT output,
Instant timestamp,
Collection<? extends BoundedWindow> windows,
PaneInfo pane);

/** Outputs a value with windowing information to a side output. */
<SideOutputT> void sideOutputWindowedValue(
TupleTag<SideOutputT> tag,
SideOutputT output,
Instant timestamp,
Collection<? extends BoundedWindow> windows,
PaneInfo pane);
}
Original file line number Diff line number Diff line change
Expand Up @@ -28,12 +28,13 @@
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
import org.apache.beam.sdk.transforms.windowing.PaneInfo;
import org.apache.beam.sdk.transforms.windowing.WindowFn;
import org.apache.beam.sdk.util.ActiveWindowSet;
import org.apache.beam.sdk.util.SideInputReader;
import org.apache.beam.sdk.util.TimeDomain;
import org.apache.beam.sdk.util.TimerInternals;
import org.apache.beam.sdk.util.TimerInternals.TimerData;
import org.apache.beam.sdk.util.Timers;
import org.apache.beam.sdk.util.WindowingInternals;
import org.apache.beam.sdk.util.WindowingStrategy;
import org.apache.beam.sdk.util.state.MergingStateAccessor;
import org.apache.beam.sdk.util.state.ReadableState;
Expand All @@ -46,6 +47,7 @@
import org.apache.beam.sdk.util.state.StateNamespaces;
import org.apache.beam.sdk.util.state.StateNamespaces.WindowNamespace;
import org.apache.beam.sdk.util.state.StateTag;
import org.apache.beam.sdk.values.PCollectionView;
import org.joda.time.Instant;

/**
Expand All @@ -62,20 +64,25 @@ public interface OnTriggerCallbacks<OutputT> {
private final StateInternals<K> stateInternals;
private final ActiveWindowSet<W> activeWindows;
private final TimerInternals timerInternals;
private final WindowingInternals<?, ?> windowingInternals;
private final SideInputReader sideInputReader;
private final PipelineOptions options;

ReduceFnContextFactory(K key, ReduceFn<K, InputT, OutputT, W> reduceFn,
WindowingStrategy<?, W> windowingStrategy, StateInternals<K> stateInternals,
ActiveWindowSet<W> activeWindows, TimerInternals timerInternals,
WindowingInternals<?, ?> windowingInternals, PipelineOptions options) {
ReduceFnContextFactory(
K key,
ReduceFn<K, InputT, OutputT, W> reduceFn,
WindowingStrategy<?, W> windowingStrategy,
StateInternals<K> stateInternals,
ActiveWindowSet<W> activeWindows,
TimerInternals timerInternals,
SideInputReader sideInputReader,
PipelineOptions options) {
this.key = key;
this.reduceFn = reduceFn;
this.windowingStrategy = windowingStrategy;
this.stateInternals = stateInternals;
this.activeWindows = activeWindows;
this.timerInternals = timerInternals;
this.windowingInternals = windowingInternals;
this.sideInputReader = sideInputReader;
this.options = options;
}

Expand All @@ -89,8 +96,14 @@ public enum StateStyle {

private StateAccessorImpl<K, W> stateAccessor(W window, StateStyle style) {
return new StateAccessorImpl<K, W>(
activeWindows, windowingStrategy.getWindowFn().windowCoder(),
stateInternals, StateContexts.createFromComponents(options, windowingInternals, window),
activeWindows,
windowingStrategy.getWindowFn().windowCoder(),
stateInternals,
stateContextFromComponents(
options,
sideInputReader,
window,
windowingStrategy.getWindowFn()),
style);
}

Expand Down Expand Up @@ -217,7 +230,7 @@ public MergingStateAccessorImpl(ActiveWindowSet<W> activeWindows, Coder<W> windo
StateInternals<K> stateInternals, StateStyle style, Collection<W> activeToBeMerged,
W mergeResult) {
super(activeWindows, windowCoder, stateInternals,
StateContexts.windowOnly(mergeResult), style);
stateContextForWindowOnly(mergeResult), style);
this.activeToBeMerged = activeToBeMerged;
}

Expand Down Expand Up @@ -262,7 +275,7 @@ static class PremergingStateAccessorImpl<K, W extends BoundedWindow>
public PremergingStateAccessorImpl(ActiveWindowSet<W> activeWindows, Coder<W> windowCoder,
StateInternals<K> stateInternals, W window) {
super(activeWindows, windowCoder, stateInternals,
StateContexts.windowOnly(window), StateStyle.RENAMED);
stateContextForWindowOnly(window), StateStyle.RENAMED);
}

Collection<W> mergingWindows() {
Expand Down Expand Up @@ -496,4 +509,52 @@ public Timers timers() {
return timers;
}
}

private static <W extends BoundedWindow> StateContext<W> stateContextFromComponents(
@Nullable final PipelineOptions options,
final SideInputReader sideInputReader,
final W mainInputWindow,
final WindowFn<?, W> windowFn) {
if (options == null) {
return StateContexts.nullContext();
} else {
return new StateContext<W>() {

@Override
public PipelineOptions getPipelineOptions() {
return options;
}

@Override
public <T> T sideInput(PCollectionView<T> view) {
return sideInputReader.get(view, windowFn.getSideInputWindow(mainInputWindow));
}

@Override
public W window() {
return mainInputWindow;
}
};
}
}

/** Returns a {@link StateContext} that only contains the state window. */
private static <W extends BoundedWindow> StateContext<W> stateContextForWindowOnly(
final W window) {
return new StateContext<W>() {
@Override
public PipelineOptions getPipelineOptions() {
throw new IllegalArgumentException(
"cannot call getPipelineOptions() in a window only context");
}
@Override
public <T> T sideInput(PCollectionView<T> view) {
throw new IllegalArgumentException("cannot call sideInput() in a window only context");
}
@Override
public W window() {
return window;
}
};
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -50,12 +50,12 @@
import org.apache.beam.sdk.util.ActiveWindowSet;
import org.apache.beam.sdk.util.MergingActiveWindowSet;
import org.apache.beam.sdk.util.NonMergingActiveWindowSet;
import org.apache.beam.sdk.util.SideInputReader;
import org.apache.beam.sdk.util.TimeDomain;
import org.apache.beam.sdk.util.TimerInternals;
import org.apache.beam.sdk.util.TimerInternals.TimerData;
import org.apache.beam.sdk.util.WindowTracing;
import org.apache.beam.sdk.util.WindowedValue;
import org.apache.beam.sdk.util.WindowingInternals;
import org.apache.beam.sdk.util.WindowingStrategy;
import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode;
import org.apache.beam.sdk.util.state.ReadableState;
Expand Down Expand Up @@ -217,15 +217,16 @@ public ReduceFnRunner(
ExecutableTriggerStateMachine triggerStateMachine,
StateInternals<K> stateInternals,
TimerInternals timerInternals,
WindowingInternals<?, KV<K, OutputT>> windowingInternals,
OutputWindowedValue<KV<K, OutputT>> outputter,
SideInputReader sideInputReader,
Aggregator<Long, Long> droppedDueToClosedWindow,
ReduceFn<K, InputT, OutputT, W> reduceFn,
PipelineOptions options) {
this.key = key;
this.timerInternals = timerInternals;
this.paneInfoTracker = new PaneInfoTracker(timerInternals);
this.stateInternals = stateInternals;
this.outputter = new OutputViaWindowingInternals<>(windowingInternals);
this.outputter = outputter;
this.droppedDueToClosedWindow = droppedDueToClosedWindow;
this.reduceFn = reduceFn;

Expand All @@ -240,8 +241,8 @@ public ReduceFnRunner(
this.activeWindows = createActiveWindowSet();

this.contextFactory =
new ReduceFnContextFactory<K, InputT, OutputT, W>(key, reduceFn, this.windowingStrategy,
stateInternals, this.activeWindows, timerInternals, windowingInternals, options);
new ReduceFnContextFactory<>(key, reduceFn, this.windowingStrategy,
stateInternals, this.activeWindows, timerInternals, sideInputReader, options);

this.watermarkHold = new WatermarkHold<>(timerInternals, windowingStrategy);
this.triggerRunner =
Expand Down Expand Up @@ -965,33 +966,4 @@ private Instant garbageCollectionTime(W window) {
return window.maxTimestamp().plus(windowingStrategy.getAllowedLateness());
}
}

/**
* An object that can output a value with all of its windowing information. This is a deliberately
* restricted subinterface of {@link WindowingInternals} to express how it is used here.
*/
private interface OutputWindowedValue<OutputT> {
void outputWindowedValue(OutputT output, Instant timestamp,
Collection<? extends BoundedWindow> windows, PaneInfo pane);
}

private static class OutputViaWindowingInternals<OutputT>
implements OutputWindowedValue<OutputT> {

private final WindowingInternals<?, OutputT> windowingInternals;

public OutputViaWindowingInternals(WindowingInternals<?, OutputT> windowingInternals) {
this.windowingInternals = windowingInternals;
}

@Override
public void outputWindowedValue(
OutputT output,
Instant timestamp,
Collection<? extends BoundedWindow> windows,
PaneInfo pane) {
windowingInternals.outputWindowedValue(output, timestamp, windows, pane);
}

}
}
Loading

0 comments on commit 4852d2e

Please sign in to comment.