Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 9ECD6200C7F for ; Wed, 19 Apr 2017 03:12:28 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 9BB09160BB2; Wed, 19 Apr 2017 01:12:28 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id C6E30160BAC for ; Wed, 19 Apr 2017 03:12:27 +0200 (CEST) Received: (qmail 88888 invoked by uid 500); 19 Apr 2017 01:12:27 -0000 Mailing-List: contact commits-help@beam.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@beam.apache.org Delivered-To: mailing list commits@beam.apache.org Received: (qmail 88751 invoked by uid 99); 19 Apr 2017 01:12:25 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 19 Apr 2017 01:12:25 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 9E6B7DFBC8; Wed, 19 Apr 2017 01:12:25 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: jkff@apache.org To: commits@beam.apache.org Date: Wed, 19 Apr 2017 01:12:26 -0000 Message-Id: In-Reply-To: <1f3496f22cd34135813c6c528028423e@git.apache.org> References: <1f3496f22cd34135813c6c528028423e@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [2/7] beam git commit: ProcessFn remembers more info about its application context archived-at: Wed, 19 Apr 2017 01:12:28 -0000 ProcessFn remembers more info about its application context Project: http://git-wip-us.apache.org/repos/asf/beam/repo Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/3fd88901 Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/3fd88901 Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/3fd88901 Branch: refs/heads/master Commit: 3fd889015afa8528801d2c35c8c9f72b944ea472 Parents: a51bdd2 Author: Eugene Kirpichov Authored: Sat Apr 15 16:39:51 2017 -0700 Committer: Eugene Kirpichov Committed: Tue Apr 18 18:02:06 2017 -0700 ---------------------------------------------------------------------- .../beam/runners/core/SplittableParDo.java | 35 +++++++++++++++----- .../beam/runners/core/SplittableParDoTest.java | 8 ++++- 2 files changed, 34 insertions(+), 9 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/beam/blob/3fd88901/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java ---------------------------------------------------------------------- diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java index 9cc965a..44db1f7 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDo.java @@ -115,7 +115,7 @@ public class SplittableParDo fn, input.getCoder(), restrictionCoder, - input.getWindowingStrategy(), + (WindowingStrategy) input.getWindowingStrategy(), parDo.getSideInputs(), parDo.getMainOutputTag(), parDo.getAdditionalOutputTags())); @@ -185,7 +185,7 @@ public class SplittableParDo private final DoFn fn; private final Coder elementCoder; private final Coder restrictionCoder; - private final WindowingStrategy windowingStrategy; + private final WindowingStrategy windowingStrategy; private final List> sideInputs; private final TupleTag mainOutputTag; private final TupleTagList additionalOutputTags; @@ -202,7 +202,7 @@ public class SplittableParDo DoFn fn, Coder elementCoder, Coder restrictionCoder, - WindowingStrategy windowingStrategy, + WindowingStrategy windowingStrategy, List> sideInputs, TupleTag mainOutputTag, TupleTagList additionalOutputTags) { @@ -234,7 +234,7 @@ public class SplittableParDo public ProcessFn newProcessFn( DoFn fn) { return new SplittableParDo.ProcessFn<>( - fn, elementCoder, restrictionCoder, windowingStrategy.getWindowFn().windowCoder()); + fn, elementCoder, restrictionCoder, windowingStrategy); } @Override @@ -351,7 +351,9 @@ public class SplittableParDo private StateTag> restrictionTag; private final DoFn fn; - private final Coder windowCoder; + private final Coder elementCoder; + private final Coder restrictionCoder; + private final WindowingStrategy inputWindowingStrategy; private transient StateInternalsFactory stateInternalsFactory; private transient TimerInternalsFactory timerInternalsFactory; @@ -364,11 +366,16 @@ public class SplittableParDo DoFn fn, Coder elementCoder, Coder restrictionCoder, - Coder windowCoder) { + WindowingStrategy inputWindowingStrategy) { this.fn = fn; - this.windowCoder = windowCoder; + this.elementCoder = elementCoder; + this.restrictionCoder = restrictionCoder; + this.inputWindowingStrategy = inputWindowingStrategy; this.elementTag = - StateTags.value("element", WindowedValue.getFullCoder(elementCoder, this.windowCoder)); + StateTags.value( + "element", + WindowedValue.getFullCoder( + elementCoder, inputWindowingStrategy.getWindowFn().windowCoder())); this.restrictionTag = StateTags.value("restriction", restrictionCoder); } @@ -389,6 +396,18 @@ public class SplittableParDo return fn; } + public Coder getElementCoder() { + return elementCoder; + } + + public Coder getRestrictionCoder() { + return restrictionCoder; + } + + public WindowingStrategy getInputWindowingStrategy() { + return inputWindowingStrategy; + } + @Setup public void setup() throws Exception { invoker = DoFnInvokers.invokerFor(fn); http://git-wip-us.apache.org/repos/asf/beam/blob/3fd88901/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java ---------------------------------------------------------------------- diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java index 2c89543..5629635 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoTest.java @@ -51,11 +51,13 @@ import org.apache.beam.sdk.transforms.splittabledofn.OffsetRange; import org.apache.beam.sdk.transforms.splittabledofn.OffsetRangeTracker; import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TimestampedValue; @@ -220,9 +222,13 @@ public class SplittableParDoTest { int maxOutputsPerBundle, Duration maxBundleDuration) throws Exception { + // The exact windowing strategy doesn't matter in this test, but it should be able to + // encode IntervalWindow's because that's what all tests here use. + WindowingStrategy windowingStrategy = + (WindowingStrategy) WindowingStrategy.of(FixedWindows.of(Duration.standardSeconds(1))); final SplittableParDo.ProcessFn processFn = new SplittableParDo.ProcessFn<>( - fn, inputCoder, restrictionCoder, IntervalWindow.getCoder()); + fn, inputCoder, restrictionCoder, windowingStrategy); this.tester = DoFnTester.of(processFn); this.timerInternals = new InMemoryTimerInternals(); this.stateInternals = new TestInMemoryStateInternals<>("dummy");