From ab617ef7b2131a5a68c791fd8f974dc93b98f5cb Mon Sep 17 00:00:00 2001 From: Vaclav Plajt Date: Mon, 14 May 2018 12:32:49 +0200 Subject: [PATCH 01/13] [BEAM-3900] Allow for Beam windowing to be set through Euphoria API. --- .../euphoria/beam/window/BeamWindowing.java | 76 +++++++++++++++++++ .../beam/window/UnsupportedWindow.java | 37 +++++++++ .../euphoria/beam/ReduceByKeyTranslator.java | 70 ++++++++++++----- .../euphoria/beam/ReduceByKeyTest.java | 21 ++++- 4 files changed, 179 insertions(+), 25 deletions(-) create mode 100644 sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/BeamWindowing.java create mode 100644 sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/UnsupportedWindow.java diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/BeamWindowing.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/BeamWindowing.java new file mode 100644 index 0000000000000..ca181e7a8a2e9 --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/BeamWindowing.java @@ -0,0 +1,76 @@ +/* + * Copyright 2016-2018 Seznam.cz, a.s. + * + * Licensed 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 cz.seznam.euphoria.beam.window; + + +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.WindowedElement; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Windowing; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.Trigger; +import org.apache.beam.sdk.transforms.windowing.WindowFn; +import org.apache.beam.sdk.values.WindowingStrategy; +import org.apache.beam.sdk.values.WindowingStrategy.AccumulationMode; + +/** + * Euphoria's {@link Windowing} which wraps Beam's {@link WindowFn}, {@link Trigger} and {@link + * WindowingStrategy.AccumulationMode} to allow Beam's widowing to be defined through Euphoria API. + */ +public class BeamWindowing implements + Windowing { + + private final WindowFn windowFn; + private final Trigger trigger; + private final WindowingStrategy.AccumulationMode accumulationMode; + + private BeamWindowing(WindowFn windowFn, Trigger beamTrigger, + AccumulationMode accumulationMode) { + this.windowFn = windowFn; + this.trigger = beamTrigger; + this.accumulationMode = accumulationMode; + } + + public static BeamWindowing of( + + WindowFn windowFn, Trigger trigger, + WindowingStrategy.AccumulationMode accumulationMode) { + return new BeamWindowing<>(windowFn, trigger, accumulationMode); + } + + + @Override + public Iterable assignWindowsToElement(WindowedElement el) { + throw new UnsupportedOperationException( + "Beam window serves as envelope, it do not supports element to window assignment."); + } + + @Override + public org.apache.beam.sdk.extensions.euphoria.core.client.triggers.Trigger getTrigger() { + throw new UnsupportedOperationException( + "Beam window serves as envelope, it do not contains Euphoria trigger."); + } + + public WindowFn getWindowFn() { + return windowFn; + } + + public AccumulationMode getAccumulationMode() { + return accumulationMode; + } + + public Trigger getBeamTrigger() { + return trigger; + } +} diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/UnsupportedWindow.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/UnsupportedWindow.java new file mode 100644 index 0000000000000..aba4f0bd50d36 --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/UnsupportedWindow.java @@ -0,0 +1,37 @@ +package cz.seznam.euphoria.beam.window; + +import java.util.Objects; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; + + +/** + * Window used as type parameter of {@link BeamWindowing}. + */ +final class UnsupportedWindow extends Window { + + private UnsupportedWindow(){ + //Do not instantiate + } + + @Override + public int compareTo(UnsupportedWindow o) { + throw new UnsupportedOperationException(); + } + + @Override + public long maxTimestamp() { + throw new UnsupportedOperationException(); + } + + @Override + public int hashCode() { + throw new UnsupportedOperationException(); + } + + @Override + public boolean equals(Object obj) { + return Objects.equals(this, obj); + } + + +} diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTranslator.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTranslator.java index 4104044038f21..e11ec673b2930 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTranslator.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTranslator.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.extensions.euphoria.beam; + +import cz.seznam.euphoria.beam.window.BeamWindowing; import java.util.stream.StreamSupport; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.IterableCoder; @@ -24,6 +26,7 @@ import org.apache.beam.sdk.extensions.euphoria.beam.window.BeamWindowFn; import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.AccumulatorProvider; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Windowing; import org.apache.beam.sdk.extensions.euphoria.core.client.functional.ReduceFunctor; import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction; import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceByKey; @@ -35,7 +38,6 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.SimpleFunction; -import org.apache.beam.sdk.transforms.windowing.AfterWatermark; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; @@ -56,24 +58,8 @@ class ReduceByKeyTranslator implements OperatorTranslator { final Coder keyCoder = context.getCoder(keyExtractor); final Coder valueCoder = context.getCoder(valueExtractor); - final PCollection input; - - // ~ apply windowing if specified - if (operator.getWindowing() == null) { - input = context.getInput(operator); - } else { - input = - context - .getInput(operator) - .apply( - operator.getName() + "::windowing", - org.apache.beam.sdk.transforms.windowing.Window.into( - BeamWindowFn.wrap(operator.getWindowing())) - // TODO: trigger - .triggering(AfterWatermark.pastEndOfWindow()) - .discardingFiredPanes() - .withAllowedLateness(context.getAllowedLateness(operator))); - } + final PCollection input = applyWindowingIfSpecified(operator, + context.getInput(operator), context); // ~ create key & value extractor final MapElements> extractor = @@ -119,11 +105,53 @@ public Pair apply(KV in) { } } + private static > + PCollection applyWindowingIfSpecified( + ReduceByKey operator, PCollection input, + BeamExecutorContext context) { + + Windowing userSpecifiedWindowing = operator.getWindowing(); + + if (userSpecifiedWindowing == null) { + return input; + } + + if (!(userSpecifiedWindowing instanceof BeamWindowing)) { + throw new IllegalStateException(String.format( + "%s class only is supported to specify windowing.", BeamWindowing.class.getSimpleName())); + } + + @SuppressWarnings("unchecked") + BeamWindowing beamWindowing = (BeamWindowing) userSpecifiedWindowing; + + @SuppressWarnings("unchecked") + org.apache.beam.sdk.transforms.windowing.Window beamWindow = + (org.apache.beam.sdk.transforms.windowing.Window) + org.apache.beam.sdk.transforms.windowing.Window.into(beamWindowing.getWindowFn()) + .triggering(beamWindowing.getBeamTrigger()); + + switch (beamWindowing.getAccumulationMode()) { + case DISCARDING_FIRED_PANES: + beamWindow = beamWindow.discardingFiredPanes(); + break; + case ACCUMULATING_FIRED_PANES: + beamWindow = beamWindow.accumulatingFiredPanes(); + break; + default: + throw new IllegalStateException( + "Unsupported accumulation mode '" + beamWindowing.getAccumulationMode() + "'"); + } + + beamWindow = beamWindow.withAllowedLateness(context.getAllowedLateness(operator)); + + return input.apply(operator.getName() + "::windowing", beamWindow); + } + private static SerializableFunction, InputT> asCombiner( ReduceFunctor reducer) { - @SuppressWarnings("unchecked") - final ReduceFunctor combiner = (ReduceFunctor) reducer; + @SuppressWarnings("unchecked") final ReduceFunctor combiner = + (ReduceFunctor) reducer; final SingleValueCollector collector = new SingleValueCollector<>(); return (Iterable input) -> { combiner.apply(StreamSupport.stream(input.spliterator(), false), collector); diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTest.java b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTest.java index 2a913ab9746a5..130316ff1e28a 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTest.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTest.java @@ -19,11 +19,11 @@ import static org.junit.Assert.assertTrue; +import cz.seznam.euphoria.beam.window.BeamWindowing; import java.time.Duration; import java.util.Arrays; import java.util.Collections; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; -import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.TimeInterval; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.WindowedElement; @@ -50,6 +50,9 @@ import org.apache.beam.sdk.extensions.euphoria.testing.DatasetAssert; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.transforms.windowing.AfterWatermark; +import org.apache.beam.sdk.transforms.windowing.FixedWindows; +import org.apache.beam.sdk.values.WindowingStrategy.AccumulationMode; import org.junit.Ignore; import org.junit.Test; @@ -76,7 +79,10 @@ public void testSimpleRBK() { ReduceByKey.of(flow.createInput(input, e -> 1000L * e)) .keyBy(i -> i % 2) .reduceBy(Sums.ofInts()) - .windowBy(Time.of(Duration.ofHours(1))) + .windowBy(BeamWindowing.of( + FixedWindows.of(org.joda.time.Duration.standardHours(1)), + AfterWatermark.pastEndOfWindow(), + AccumulationMode.DISCARDING_FIRED_PANES)) .output() .persist(output); @@ -118,11 +124,15 @@ public void testEventTime() { ListDataSink> sink = ListDataSink.get(); Dataset> input = flow.createInput(source); input = AssignEventTime.of(input).using(Pair::getSecond).output(); + ReduceByKey.of(input) .keyBy(Pair::getFirst) .valueBy(e -> 1L) .combineBy(Sums.ofLongs()) - .windowBy(Time.of(Duration.ofSeconds(1))) + .windowBy(BeamWindowing.of( + FixedWindows.of(org.joda.time.Duration.standardSeconds(1)), + AfterWatermark.pastEndOfWindow(), + AccumulationMode.DISCARDING_FIRED_PANES)) .output() .persist(sink); @@ -172,7 +182,10 @@ public void testElementTimestamp() { .keyBy(e -> "", TypeHint.ofString()) .valueBy(Pair::getFirst, TypeHint.ofInt()) .combineBy(Sums.ofInts(), TypeHint.ofInt()) - .windowBy(Time.of(Duration.ofSeconds(5))) + .windowBy(BeamWindowing.of( + FixedWindows.of(org.joda.time.Duration.standardSeconds(5)), + AfterWatermark.pastEndOfWindow(), + AccumulationMode.DISCARDING_FIRED_PANES)) .output(); // ~ now use a custom windowing with a trigger which does // the assertions subject to this test (use RSBK which has to From b99c6aaa4ab1924156e2850b68da64c7e19ee524 Mon Sep 17 00:00:00 2001 From: Vaclav Plajt Date: Mon, 14 May 2018 14:05:40 +0200 Subject: [PATCH 02/13] [BEAM-3900] Missing licence header added. --- .../euphoria/beam/window/UnsupportedWindow.java | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/UnsupportedWindow.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/UnsupportedWindow.java index aba4f0bd50d36..b4594947cdbdf 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/UnsupportedWindow.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/UnsupportedWindow.java @@ -1,3 +1,20 @@ +/* + * 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 cz.seznam.euphoria.beam.window; import java.util.Objects; From 3aa7e85bcbb1d33a9afde2d1fd54c66ba06376a7 Mon Sep 17 00:00:00 2001 From: Vaclav Plajt Date: Mon, 14 May 2018 14:50:43 +0200 Subject: [PATCH 03/13] [BEAM-4294] Join translator empty shell added. --- .../seznam/euphoria/beam/JoinTranslator.java | 25 +++++++++++++++++++ .../euphoria/beam/join/package-info.java | 22 ++++++++++++++++ .../euphoria/beam/FlowTranslator.java | 1 + 3 files changed, 48 insertions(+) create mode 100644 sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/JoinTranslator.java create mode 100644 sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/package-info.java diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/JoinTranslator.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/JoinTranslator.java new file mode 100644 index 0000000000000..922a2af79e264 --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/JoinTranslator.java @@ -0,0 +1,25 @@ +package cz.seznam.euphoria.beam; + +import cz.seznam.euphoria.core.client.dataset.windowing.Window; +import cz.seznam.euphoria.core.client.operator.Join; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; + +public class JoinTranslator implements OperatorTranslator { + + @Override + @SuppressWarnings("unchecked") + public PCollection translate(Join operator, BeamExecutorContext context) { + return doTranslate(operator, context); + } + + + public > PCollection>> + doTranslate(Join operator, BeamExecutorContext context) { + + + + + throw new UnsupportedOperationException("Not supported yet"); + } +} diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/package-info.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/package-info.java new file mode 100644 index 0000000000000..e47d9dadca8b4 --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * {@link cz.seznam.euphoria.core.client.operator.Join} translation centered classes. + */ +package cz.seznam.euphoria.beam.join; diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/FlowTranslator.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/FlowTranslator.java index e03a65e424a89..0d4ec5394f348 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/FlowTranslator.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/FlowTranslator.java @@ -54,6 +54,7 @@ class FlowTranslator { // extended operators translators.put(ReduceByKey.class, new ReduceByKeyTranslator()); translators.put(ReduceStateByKey.class, new ReduceStateByKeyTranslator()); + translators.put(JoinTranslator.class, new JoinTranslator()); } static Pipeline toPipeline( From 8c2b4eec2e00c7cdd49074a5d0208269277641f0 Mon Sep 17 00:00:00 2001 From: Vaclav Plajt Date: Tue, 15 May 2018 10:55:20 +0200 Subject: [PATCH 04/13] [BEAM-4294] Crude Join operator translation implemented. Still work in progress. --- .../seznam/euphoria/beam/JoinTranslator.java | 85 ++++++++++++++++++- .../euphoria/beam/common/InputToKvDoFn.java | 22 +++++ .../euphoria/beam/join/InnerJoinFn.java | 44 ++++++++++ .../cz/seznam/euphoria/beam/join/JoinFn.java | 28 ++++++ .../seznam/euphoria/beam/join/LeftJoinFn.java | 38 +++++++++ .../euphoria/beam/window/WindowingUtils.java | 53 ++++++++++++ .../euphoria/beam/ReduceByKeyTranslator.java | 48 +---------- 7 files changed, 270 insertions(+), 48 deletions(-) create mode 100644 sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/common/InputToKvDoFn.java create mode 100644 sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/InnerJoinFn.java create mode 100644 sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/JoinFn.java create mode 100644 sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/LeftJoinFn.java create mode 100644 sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/WindowingUtils.java diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/JoinTranslator.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/JoinTranslator.java index 922a2af79e264..ccfd66118a2a3 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/JoinTranslator.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/JoinTranslator.java @@ -1,9 +1,24 @@ package cz.seznam.euphoria.beam; +import cz.seznam.euphoria.beam.io.KryoCoder; +import cz.seznam.euphoria.beam.join.InnerJoinFn; +import cz.seznam.euphoria.beam.common.InputToKvDoFn; +import cz.seznam.euphoria.beam.join.JoinFn; +import cz.seznam.euphoria.beam.window.WindowingUtils; import cz.seznam.euphoria.core.client.dataset.windowing.Window; +import cz.seznam.euphoria.core.client.functional.UnaryFunction; import cz.seznam.euphoria.core.client.operator.Join; +import cz.seznam.euphoria.core.client.util.Pair; +import java.util.List; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.join.CoGbkResult; +import org.apache.beam.sdk.transforms.join.CoGroupByKey; +import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.TupleTag; public class JoinTranslator implements OperatorTranslator { @@ -14,12 +29,76 @@ public PCollection translate(Join operator, BeamExecutorContext context) { } - public > PCollection>> - doTranslate(Join operator, BeamExecutorContext context) { + public > PCollection> + doTranslate(Join operator, BeamExecutorContext context) { + Coder keyCoder = context.getCoder(operator.getLeftKeyExtractor()); + // get input data-sets transformed to Pcollections> + List> inputs = context.getInputs(operator); + //TODO test left/right side indexes ! + PCollection> leftKvInput = getKVInputCollection(inputs.get(0), + operator.getLeftKeyExtractor(), + keyCoder, new KryoCoder<>(), "::extract-keys-left"); - throw new UnsupportedOperationException("Not supported yet"); + PCollection> rightKvInput = getKVInputCollection(inputs.get(1), + operator.getRightKeyExtractor(), + keyCoder, new KryoCoder<>(), "::extract-keys-right"); + + // GoGroupByKey collections + TupleTag leftTag = new TupleTag<>(); + TupleTag rightTag = new TupleTag<>(); + + PCollection> coGrouped = KeyedPCollectionTuple + .of(leftTag, leftKvInput) + .and(rightTag, rightKvInput) + .apply("::co-group-by-key", CoGroupByKey.create()); + + coGrouped = WindowingUtils + .applyWindowingIfSpecified(operator, coGrouped, context.getAllowedLateness(operator)); + + // Join + JoinFn joinFn = chooseJoinFn(operator, leftTag, rightTag); + + return coGrouped.apply(joinFn.getFnName(), ParDo.of(joinFn)); + } + + private PCollection> getKVInputCollection( + PCollection inputPCollection, + UnaryFunction keyExtractor, + Coder keyCoder, Coder valueCoder, String transformName) { + + @SuppressWarnings("unchecked") + PCollection typedInput = (PCollection) inputPCollection; + typedInput.setCoder(valueCoder); + + PCollection> leftKvInput = + typedInput.apply(transformName, ParDo.of(new InputToKvDoFn<>(keyExtractor))); + leftKvInput.setCoder(KvCoder.of(keyCoder, valueCoder)); + + return leftKvInput; + } + + private > JoinFn chooseJoinFn( + + Join operator, TupleTag leftTag, + TupleTag rightTag) { + // choose right ParDo to do LEFT, RIGHT, INNER or OUTER join + JoinFn joinFn; + switch (operator.getType()) { + case INNER: + joinFn = new InnerJoinFn<>(operator.getJoiner(), leftTag, rightTag); + break; + case RIGHT: + case LEFT: + case FULL: + default: + throw new UnsupportedOperationException(String.format( + "Cannot translate Euphoria '%s' operator to Beam transformations." + + " Given join type '%s' is not supported.", + Join.class.getSimpleName(), operator.getType())); + } + return joinFn; } } diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/common/InputToKvDoFn.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/common/InputToKvDoFn.java new file mode 100644 index 0000000000000..7c750a5450edb --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/common/InputToKvDoFn.java @@ -0,0 +1,22 @@ +package cz.seznam.euphoria.beam.common; + +import cz.seznam.euphoria.core.client.functional.UnaryFunction; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.values.KV; + +public class InputToKvDoFn extends DoFn> { + + private final UnaryFunction keyExtractor; + + public InputToKvDoFn(UnaryFunction leftKeyExtractor) { + this.keyExtractor = leftKeyExtractor; + } + + @ProcessElement + public void processElement(ProcessContext c) { + InputT element = c.element(); + K key = keyExtractor.apply(element); + c.output(KV.of(key, element)); + } + +} diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/InnerJoinFn.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/InnerJoinFn.java new file mode 100644 index 0000000000000..0c91a51a722d3 --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/InnerJoinFn.java @@ -0,0 +1,44 @@ +package cz.seznam.euphoria.beam.join; + +import cz.seznam.euphoria.beam.SingleValueCollector; +import cz.seznam.euphoria.core.client.functional.BinaryFunctor; +import cz.seznam.euphoria.core.client.util.Pair; +import org.apache.beam.sdk.transforms.join.CoGbkResult; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.TupleTag; + +public class InnerJoinFn extends JoinFn { + + public InnerJoinFn( + BinaryFunctor functor, + TupleTag leftTag, + TupleTag rightTag) { + super(functor, leftTag, rightTag); + } + + @Override + public void processElement(ProcessContext c) { + + KV element = c.element(); + CoGbkResult value = element.getValue(); + K key = element.getKey(); + + Iterable leftSideIter = value.getAll(leftTag); + Iterable rightSideIter = value.getAll(rightTag); + + SingleValueCollector outCollector = new SingleValueCollector<>(); + + for (LeftT leftItem : leftSideIter) { + for (RightT rightItem : rightSideIter) { + joiner.apply(leftItem, rightItem, outCollector); + c.output(Pair.of(key, outCollector.get())); + } + } + } + + @Override + public String getFnName() { + return "::inner-join"; + } + +} diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/JoinFn.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/JoinFn.java new file mode 100644 index 0000000000000..358ce54714579 --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/JoinFn.java @@ -0,0 +1,28 @@ +package cz.seznam.euphoria.beam.join; + +import cz.seznam.euphoria.core.client.functional.BinaryFunctor; +import cz.seznam.euphoria.core.client.util.Pair; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.join.CoGbkResult; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.TupleTag; + +public abstract class JoinFn extends DoFn, Pair> { + + protected final BinaryFunctor joiner; + protected final TupleTag leftTag; + protected final TupleTag rightTag; + + protected JoinFn( + BinaryFunctor joiner, + TupleTag leftTag, TupleTag rightTag) { + this.joiner = joiner; + this.leftTag = leftTag; + this.rightTag = rightTag; + } + + @ProcessElement + public abstract void processElement(ProcessContext c); + + public abstract String getFnName(); +} diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/LeftJoinFn.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/LeftJoinFn.java new file mode 100644 index 0000000000000..092428b0d91a4 --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/LeftJoinFn.java @@ -0,0 +1,38 @@ +package cz.seznam.euphoria.beam.join; + +import cz.seznam.euphoria.beam.SingleValueCollector; +import cz.seznam.euphoria.core.client.functional.BinaryFunctor; +import org.apache.beam.sdk.transforms.join.CoGbkResult; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.TupleTag; + +//TODO implement +public class LeftJoinFn /*extends JoinFn */{ +/* + protected LeftJoinFn( + BinaryFunctor joiner, + TupleTag leftTag, + TupleTag rightTag) { + super(joiner, leftTag, rightTag); + } + + @Override + public void processElement(ProcessContext c) { + + KV element = c.element(); + CoGbkResult value = element.getValue(); + + Iterable leftSideIter = value.getAll(leftTag); + Iterable rightSIdeIter = value.getAll(rightTag); + + SingleValueCollector outCollector = new SingleValueCollector<>(); + + + } + + @Override + public String getFnName() { + return "::left-join"; + } + */ +} diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/WindowingUtils.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/WindowingUtils.java new file mode 100644 index 0000000000000..69cdd11c143bd --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/WindowingUtils.java @@ -0,0 +1,53 @@ +package cz.seznam.euphoria.beam.window; + +import cz.seznam.euphoria.core.client.dataset.windowing.Window; +import cz.seznam.euphoria.core.client.dataset.windowing.Windowing; +import cz.seznam.euphoria.core.client.operator.WindowWiseOperator; +import org.apache.beam.sdk.values.PCollection; +import org.joda.time.Duration; + +public class WindowingUtils { + + public static > + PCollection applyWindowingIfSpecified( + WindowWiseOperator operator, + PCollection input, + Duration allowedLateness) { + + Windowing userSpecifiedWindowing = operator.getWindowing(); + + if (userSpecifiedWindowing == null) { + return input; + } + + if (!(userSpecifiedWindowing instanceof BeamWindowing)) { + throw new IllegalStateException(String.format( + "%s class only is supported to specify windowing.", BeamWindowing.class.getSimpleName())); + } + + @SuppressWarnings("unchecked") + BeamWindowing beamWindowing = (BeamWindowing) userSpecifiedWindowing; + + @SuppressWarnings("unchecked") + org.apache.beam.sdk.transforms.windowing.Window beamWindow = + (org.apache.beam.sdk.transforms.windowing.Window) + org.apache.beam.sdk.transforms.windowing.Window.into(beamWindowing.getWindowFn()) + .triggering(beamWindowing.getBeamTrigger()); + + switch (beamWindowing.getAccumulationMode()) { + case DISCARDING_FIRED_PANES: + beamWindow = beamWindow.discardingFiredPanes(); + break; + case ACCUMULATING_FIRED_PANES: + beamWindow = beamWindow.accumulatingFiredPanes(); + break; + default: + throw new IllegalStateException( + "Unsupported accumulation mode '" + beamWindowing.getAccumulationMode() + "'"); + } + + beamWindow = beamWindow.withAllowedLateness(allowedLateness); + + return input.apply(operator.getName() + "::windowing", beamWindow); + } +} diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTranslator.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTranslator.java index e11ec673b2930..c22492e8591d8 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTranslator.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTranslator.java @@ -18,7 +18,7 @@ package org.apache.beam.sdk.extensions.euphoria.beam; -import cz.seznam.euphoria.beam.window.BeamWindowing; +import cz.seznam.euphoria.beam.window.WindowingUtils; import java.util.stream.StreamSupport; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.IterableCoder; @@ -58,8 +58,8 @@ class ReduceByKeyTranslator implements OperatorTranslator { final Coder keyCoder = context.getCoder(keyExtractor); final Coder valueCoder = context.getCoder(valueExtractor); - final PCollection input = applyWindowingIfSpecified(operator, - context.getInput(operator), context); + final PCollection input = WindowingUtils.applyWindowingIfSpecified(operator, + context.getInput(operator), context.getAllowedLateness(operator)); // ~ create key & value extractor final MapElements> extractor = @@ -105,48 +105,6 @@ public Pair apply(KV in) { } } - private static > - PCollection applyWindowingIfSpecified( - ReduceByKey operator, PCollection input, - BeamExecutorContext context) { - - Windowing userSpecifiedWindowing = operator.getWindowing(); - - if (userSpecifiedWindowing == null) { - return input; - } - - if (!(userSpecifiedWindowing instanceof BeamWindowing)) { - throw new IllegalStateException(String.format( - "%s class only is supported to specify windowing.", BeamWindowing.class.getSimpleName())); - } - - @SuppressWarnings("unchecked") - BeamWindowing beamWindowing = (BeamWindowing) userSpecifiedWindowing; - - @SuppressWarnings("unchecked") - org.apache.beam.sdk.transforms.windowing.Window beamWindow = - (org.apache.beam.sdk.transforms.windowing.Window) - org.apache.beam.sdk.transforms.windowing.Window.into(beamWindowing.getWindowFn()) - .triggering(beamWindowing.getBeamTrigger()); - - switch (beamWindowing.getAccumulationMode()) { - case DISCARDING_FIRED_PANES: - beamWindow = beamWindow.discardingFiredPanes(); - break; - case ACCUMULATING_FIRED_PANES: - beamWindow = beamWindow.accumulatingFiredPanes(); - break; - default: - throw new IllegalStateException( - "Unsupported accumulation mode '" + beamWindowing.getAccumulationMode() + "'"); - } - - beamWindow = beamWindow.withAllowedLateness(context.getAllowedLateness(operator)); - - return input.apply(operator.getName() + "::windowing", beamWindow); - } - private static SerializableFunction, InputT> asCombiner( ReduceFunctor reducer) { From cd31e9f7babf6ac5e513b29052f2a6aae0d71aed Mon Sep 17 00:00:00 2001 From: Vaclav Plajt Date: Tue, 15 May 2018 15:20:41 +0200 Subject: [PATCH 05/13] [BEAM-4294] Inner join translation implemented. --- .../seznam/euphoria/beam/JoinTranslator.java | 12 ++- .../euphoria/beam/common/InputToKvDoFn.java | 4 + .../euphoria/beam/common/package-info.java | 22 +++++ .../euphoria/beam/join/InnerJoinFn.java | 3 + .../cz/seznam/euphoria/beam/join/JoinFn.java | 13 ++- .../seznam/euphoria/beam/join/LeftJoinFn.java | 7 +- .../euphoria/beam/window/WindowingUtils.java | 10 ++- .../euphoria/beam/FlowTranslator.java | 4 +- .../beam/ReduceStateByKeyTranslator.java | 2 +- .../cz/seznam/euphoria/beam/JoinTest.java | 82 +++++++++++++++++++ .../cz/seznam/euphoria/beam/TestUtils.java | 17 ++++ .../euphoria/beam/ReduceByKeyTest.java | 13 +-- .../beam/testkit/BeamExecutorProvider.java | 1 + 13 files changed, 168 insertions(+), 22 deletions(-) create mode 100644 sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/common/package-info.java create mode 100644 sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/JoinTest.java create mode 100644 sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/TestUtils.java diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/JoinTranslator.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/JoinTranslator.java index ccfd66118a2a3..4fb88d9bcb6e2 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/JoinTranslator.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/JoinTranslator.java @@ -1,8 +1,8 @@ package cz.seznam.euphoria.beam; +import cz.seznam.euphoria.beam.common.InputToKvDoFn; import cz.seznam.euphoria.beam.io.KryoCoder; import cz.seznam.euphoria.beam.join.InnerJoinFn; -import cz.seznam.euphoria.beam.common.InputToKvDoFn; import cz.seznam.euphoria.beam.join.JoinFn; import cz.seznam.euphoria.beam.window.WindowingUtils; import cz.seznam.euphoria.core.client.dataset.windowing.Window; @@ -20,6 +20,10 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TupleTag; + +/** + * {@link OperatorTranslator Translator } for Euphoria {@link Join} operator. + */ public class JoinTranslator implements OperatorTranslator { @Override @@ -80,11 +84,11 @@ private PCollection> getKVInputCollection( return leftKvInput; } - private > JoinFn chooseJoinFn( - + private > JoinFn + chooseJoinFn( Join operator, TupleTag leftTag, TupleTag rightTag) { - // choose right ParDo to do LEFT, RIGHT, INNER or OUTER join + JoinFn joinFn; switch (operator.getType()) { case INNER: diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/common/InputToKvDoFn.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/common/InputToKvDoFn.java index 7c750a5450edb..dbac09e738fec 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/common/InputToKvDoFn.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/common/InputToKvDoFn.java @@ -4,6 +4,10 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.values.KV; +/** + * {@link DoFn} which takes input elements and transforms them to {@link KV} using given key + * extractor. + */ public class InputToKvDoFn extends DoFn> { private final UnaryFunction keyExtractor; diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/common/package-info.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/common/package-info.java new file mode 100644 index 0000000000000..c238720a5c4d9 --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/common/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * A set of commonly used classes enabling some code reuse. + */ +package cz.seznam.euphoria.beam.common; diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/InnerJoinFn.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/InnerJoinFn.java index 0c91a51a722d3..9aebb8b01d0b9 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/InnerJoinFn.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/InnerJoinFn.java @@ -7,6 +7,9 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +/** + * Inner join implementation of {@link JoinFn}. + */ public class InnerJoinFn extends JoinFn { public InnerJoinFn( diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/JoinFn.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/JoinFn.java index 358ce54714579..bee8d74ffa9c1 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/JoinFn.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/JoinFn.java @@ -7,7 +7,16 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; -public abstract class JoinFn extends DoFn, Pair> { +/** + * Abstract base for joint implementations. + * + * @param type of left-side elements + * @param type of right-side elements + * @param key type + * @param type of output elements + */ +public abstract class JoinFn extends + DoFn, Pair> { protected final BinaryFunctor joiner; protected final TupleTag leftTag; @@ -24,5 +33,5 @@ protected JoinFn( @ProcessElement public abstract void processElement(ProcessContext c); - public abstract String getFnName(); + public abstract String getFnName(); } diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/LeftJoinFn.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/LeftJoinFn.java index 092428b0d91a4..e452b054e4545 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/LeftJoinFn.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/LeftJoinFn.java @@ -1,12 +1,15 @@ package cz.seznam.euphoria.beam.join; - +/* import cz.seznam.euphoria.beam.SingleValueCollector; import cz.seznam.euphoria.core.client.functional.BinaryFunctor; import org.apache.beam.sdk.transforms.join.CoGbkResult; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +*/ -//TODO implement +/** + * Left Join implementation of {@link JoinFn}. + */ public class LeftJoinFn /*extends JoinFn */{ /* protected LeftJoinFn( diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/WindowingUtils.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/WindowingUtils.java index 69cdd11c143bd..39ee815bbee27 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/WindowingUtils.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/WindowingUtils.java @@ -3,9 +3,13 @@ import cz.seznam.euphoria.core.client.dataset.windowing.Window; import cz.seznam.euphoria.core.client.dataset.windowing.Windowing; import cz.seznam.euphoria.core.client.operator.WindowWiseOperator; +import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.values.PCollection; import org.joda.time.Duration; +/** + * Collection of method helpful when dealing with windowing translations. + */ public class WindowingUtils { public static > @@ -30,9 +34,9 @@ PCollection applyWindowingIfSpecified( @SuppressWarnings("unchecked") org.apache.beam.sdk.transforms.windowing.Window beamWindow = - (org.apache.beam.sdk.transforms.windowing.Window) - org.apache.beam.sdk.transforms.windowing.Window.into(beamWindowing.getWindowFn()) - .triggering(beamWindowing.getBeamTrigger()); + org.apache.beam.sdk.transforms.windowing.Window + .into((WindowFn) beamWindowing.getWindowFn()) + .triggering(beamWindowing.getBeamTrigger()); switch (beamWindowing.getAccumulationMode()) { case DISCARDING_FIRED_PANES: diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/FlowTranslator.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/FlowTranslator.java index 0d4ec5394f348..e8d9af270c62e 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/FlowTranslator.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/FlowTranslator.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.extensions.euphoria.beam; +import cz.seznam.euphoria.beam.JoinTranslator; import java.util.IdentityHashMap; import java.util.Map; import org.apache.beam.sdk.Pipeline; @@ -25,6 +26,7 @@ import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; import org.apache.beam.sdk.extensions.euphoria.core.client.io.DataSink; import org.apache.beam.sdk.extensions.euphoria.core.client.operator.FlatMap; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Join; import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Operator; import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceByKey; import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceStateByKey; @@ -54,7 +56,7 @@ class FlowTranslator { // extended operators translators.put(ReduceByKey.class, new ReduceByKeyTranslator()); translators.put(ReduceStateByKey.class, new ReduceStateByKeyTranslator()); - translators.put(JoinTranslator.class, new JoinTranslator()); + translators.put(Join.class, new JoinTranslator()); } static Pipeline toPipeline( diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceStateByKeyTranslator.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceStateByKeyTranslator.java index 7e3ae7718f922..b29ea5351bdc8 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceStateByKeyTranslator.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceStateByKeyTranslator.java @@ -65,6 +65,6 @@ public PCollection translate(ReduceStateByKey operator, BeamExecutorContext c .setCoder(new KryoCoder<>()); } */ - throw new UnsupportedOperationException("Not supported yet"); + throw new UnsupportedOperationException("ReduceStateByKy is not supported yet."); } } diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/JoinTest.java b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/JoinTest.java new file mode 100644 index 0000000000000..d5ef67ddd39f9 --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/JoinTest.java @@ -0,0 +1,82 @@ +package cz.seznam.euphoria.beam; + +import static java.util.Arrays.asList; + +import cz.seznam.euphoria.core.client.flow.Flow; +import cz.seznam.euphoria.core.client.functional.BinaryFunctor; +import cz.seznam.euphoria.core.client.io.ListDataSink; +import cz.seznam.euphoria.core.client.io.ListDataSource; +import cz.seznam.euphoria.core.client.operator.Join; +import cz.seznam.euphoria.core.client.util.Pair; +import cz.seznam.euphoria.testing.DatasetAssert; +import org.junit.Test; + +/** + * Simple test suite for Join operator. + */ +public class JoinTest { + + @Test + public void simpleInnerJoinTest() { + final Flow flow = Flow.create(); + + ListDataSource> left = + ListDataSource.bounded( + asList( + Pair.of(1, "L v1"), Pair.of(1, "L v2"), + Pair.of(2, "L v1"), Pair.of(2, "L v2"), + Pair.of(3, "L v1") + )); + + ListDataSource> right = + ListDataSource.bounded( + asList( + Pair.of(1, 1), Pair.of(1, 10), + Pair.of(2, 20), + Pair.of(4, 40) + )); + + ListDataSink>> output = ListDataSink.get(); + + BinaryFunctor, Pair, Pair> joiner = + (l, r, c) -> c.collect(Pair.of(l.getSecond(), r.getSecond())); + + Join.of(flow.createInput(left), flow.createInput(right)) + .by(Pair::getFirst, Pair::getFirst) + .using(joiner) + .output() + .persist(output); + + BeamExecutor executor = TestUtils.createExecutor(); + executor.execute(flow); + + DatasetAssert.unorderedEquals(output.getOutputs(), + Pair.of(1, Pair.of("L v1", 1)), Pair.of(1, Pair.of("L v1", 10)), + Pair.of(1, Pair.of("L v2", 1)), Pair.of(1, Pair.of("L v2", 10)), + + Pair.of(2, Pair.of("L v1", 20)), Pair.of(2, Pair.of("L v2", 20)) + ); + + } + +// public void someFutureTest() { +// +// ListDataSource> left = +// ListDataSource.unbounded( +// asList( +// Pair.of(1, "L v1"), Pair.of(1, "L v2"), Pair.of(1, "L v3"), +// Pair.of(2, "L v1"), Pair.of(2, "L v2"), +// Pair.of(3, "L v1") +// )); +// +// ListDataSource> right = +// ListDataSource.unbounded( +// asList( +// Pair.of(1, 1), Pair.of(1, 10), +// Pair.of(2, 20), +// Pair.of(3, 30), Pair.of(3, 300), Pair.of(3, 3000) +// )); +// } + + +} diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/TestUtils.java b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/TestUtils.java new file mode 100644 index 0000000000000..1cc188ca0e13a --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/TestUtils.java @@ -0,0 +1,17 @@ +package cz.seznam.euphoria.beam; + +import java.time.Duration; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; + +/** + * Collection of methods reused among tests. + */ +public class TestUtils { + + static BeamExecutor createExecutor() { + String[] args = {"--runner=DirectRunner"}; + PipelineOptions options = PipelineOptionsFactory.fromArgs(args).as(PipelineOptions.class); + return new BeamExecutor(options).withAllowedLateness(Duration.ofHours(1)); + } +} diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTest.java b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTest.java index 130316ff1e28a..1ff11adc76c15 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTest.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTest.java @@ -19,6 +19,7 @@ import static org.junit.Assert.assertTrue; +import cz.seznam.euphoria.beam.TestUtils; import cz.seznam.euphoria.beam.window.BeamWindowing; import java.time.Duration; import java.util.Arrays; @@ -61,12 +62,6 @@ */ public class ReduceByKeyTest { - private BeamExecutor createExecutor() { - String[] args = {"--runner=DirectRunner"}; - PipelineOptions options = PipelineOptionsFactory.fromArgs(args).as(PipelineOptions.class); - return new BeamExecutor(options).withAllowedLateness(Duration.ofHours(1)); - } - @Test public void testSimpleRBK() { final Flow flow = Flow.create(); @@ -86,7 +81,7 @@ public void testSimpleRBK() { .output() .persist(output); - BeamExecutor executor = createExecutor(); + BeamExecutor executor = TestUtils.createExecutor(); executor.execute(flow); DatasetAssert.unorderedEquals(output.getOutputs(), Pair.of(0, 8), Pair.of(1, 7)); @@ -136,7 +131,7 @@ public void testEventTime() { .output() .persist(sink); - BeamExecutor executor = createExecutor(); + BeamExecutor executor = TestUtils.createExecutor(); executor.execute(flow); DatasetAssert.unorderedEquals( @@ -205,7 +200,7 @@ public void testElementTimestamp() { .output() .persist(sink); - createExecutor().execute(flow); + TestUtils.createExecutor().execute(flow); DatasetAssert.unorderedEquals(sink.getOutputs(), 4, 6); } diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/testkit/BeamExecutorProvider.java b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/testkit/BeamExecutorProvider.java index ddac9f5af937f..928a059e610f8 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/testkit/BeamExecutorProvider.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/testkit/BeamExecutorProvider.java @@ -30,6 +30,7 @@ */ public interface BeamExecutorProvider extends ExecutorProvider { + @Override default ExecutorEnvironment newExecutorEnvironment() throws Exception { final String[] args = {"--runner=DirectRunner"}; final PipelineOptions options = PipelineOptionsFactory.fromArgs(args).as(PipelineOptions.class); From baff3f37014050f905edd7bb7fe9c0d8a0484b27 Mon Sep 17 00:00:00 2001 From: Vaclav Plajt Date: Wed, 16 May 2018 10:02:31 +0200 Subject: [PATCH 06/13] [BEAM-4294] Left, right and full joins implemeted. --- .../euphoria/euphoria-beam/build.gradle | 3 +- .../seznam/euphoria/beam/JoinTranslator.java | 17 +- .../seznam/euphoria/beam/join/FullJoinFn.java | 60 +++++++ .../euphoria/beam/join/LeftOuterJoinFn.java | 55 ++++++ ...{LeftJoinFn.java => RightOuterJoinFn.java} | 27 ++- .../cz/seznam/euphoria/beam/JoinTest.java | 160 ++++++++++++++++-- 6 files changed, 293 insertions(+), 29 deletions(-) create mode 100644 sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/FullJoinFn.java create mode 100644 sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/LeftOuterJoinFn.java rename sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/{LeftJoinFn.java => RightOuterJoinFn.java} (55%) diff --git a/sdks/java/extensions/euphoria/euphoria-beam/build.gradle b/sdks/java/extensions/euphoria/euphoria-beam/build.gradle index 2d5cb36e453ef..df0a31fef894c 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/build.gradle +++ b/sdks/java/extensions/euphoria/euphoria-beam/build.gradle @@ -31,9 +31,10 @@ dependencies { shadow 'com.google.code.findbugs:annotations:3.0.1' testCompile project(':beam-sdks-java-extensions-euphoria-operator-testkit') testCompile project(':beam-sdks-java-extensions-euphoria-testing') - testCompile project(':beam-runners-direct-java') + //testCompile project(':beam-runners-direct-java') testCompile library.java.slf4j_api testCompile library.java.hamcrest_core + testCompile "org.apache.beam:beam-runners-direct-java:2.4.0" } diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/JoinTranslator.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/JoinTranslator.java index 4fb88d9bcb6e2..003eb60b6ba1b 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/JoinTranslator.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/JoinTranslator.java @@ -2,10 +2,14 @@ import cz.seznam.euphoria.beam.common.InputToKvDoFn; import cz.seznam.euphoria.beam.io.KryoCoder; +import cz.seznam.euphoria.beam.join.FullJoinFn; import cz.seznam.euphoria.beam.join.InnerJoinFn; import cz.seznam.euphoria.beam.join.JoinFn; +import cz.seznam.euphoria.beam.join.LeftOuterJoinFn; +import cz.seznam.euphoria.beam.join.RightOuterJoinFn; import cz.seznam.euphoria.beam.window.WindowingUtils; import cz.seznam.euphoria.core.client.dataset.windowing.Window; +import cz.seznam.euphoria.core.client.functional.BinaryFunctor; import cz.seznam.euphoria.core.client.functional.UnaryFunction; import cz.seznam.euphoria.core.client.operator.Join; import cz.seznam.euphoria.core.client.util.Pair; @@ -90,13 +94,22 @@ private PCollection> getKVInputCollection( TupleTag rightTag) { JoinFn joinFn; + BinaryFunctor joiner = operator.getJoiner(); + switch (operator.getType()) { case INNER: - joinFn = new InnerJoinFn<>(operator.getJoiner(), leftTag, rightTag); + joinFn = new InnerJoinFn<>(joiner, leftTag, rightTag); break; - case RIGHT: case LEFT: + joinFn = new LeftOuterJoinFn<>(joiner, leftTag, rightTag); + break; + case RIGHT: + joinFn = new RightOuterJoinFn<>(joiner, leftTag, rightTag); + break; case FULL: + joinFn = new FullJoinFn<>(joiner, leftTag, rightTag); + break; + default: throw new UnsupportedOperationException(String.format( "Cannot translate Euphoria '%s' operator to Beam transformations." diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/FullJoinFn.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/FullJoinFn.java new file mode 100644 index 0000000000000..b04faa971d93f --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/FullJoinFn.java @@ -0,0 +1,60 @@ +package cz.seznam.euphoria.beam.join; + +import cz.seznam.euphoria.beam.SingleValueCollector; +import cz.seznam.euphoria.core.client.functional.BinaryFunctor; +import cz.seznam.euphoria.core.client.util.Pair; +import org.apache.beam.sdk.transforms.join.CoGbkResult; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.TupleTag; + +/** + * Full join implementation of {@link JoinFn}. + */ +public class FullJoinFn extends JoinFn { + + public FullJoinFn(BinaryFunctor joiner, TupleTag leftTag, + TupleTag rightTag) { + super(joiner, leftTag, rightTag); + } + + @Override + public void processElement(ProcessContext c) { + + KV element = c.element(); + CoGbkResult value = element.getValue(); + K key = element.getKey(); + + Iterable leftSideIter = value.getAll(leftTag); + Iterable rightSIdeIter = value.getAll(rightTag); + + SingleValueCollector outCollector = new SingleValueCollector<>(); + + boolean leftHasValues = leftSideIter.iterator().hasNext(); + boolean rightHasValues = rightSIdeIter.iterator().hasNext(); + + if (leftHasValues && rightHasValues) { + for (RightT rightValue : rightSIdeIter) { + for (LeftT leftValue : leftSideIter) { + joiner.apply(leftValue, rightValue, outCollector); + c.output(Pair.of(key, outCollector.get())); + } + } + } else if (leftHasValues && !rightHasValues) { + for (LeftT leftValue : leftSideIter) { + joiner.apply(leftValue, null, outCollector); + c.output(Pair.of(key, outCollector.get())); + } + } else if (!leftHasValues && rightHasValues) { + for (RightT rightValue : rightSIdeIter) { + joiner.apply(null, rightValue, outCollector); + c.output(Pair.of(key, outCollector.get())); + } + } + + } + + @Override + public String getFnName() { + return "::full-join"; + } +} diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/LeftOuterJoinFn.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/LeftOuterJoinFn.java new file mode 100644 index 0000000000000..e0d413f584c9f --- /dev/null +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/LeftOuterJoinFn.java @@ -0,0 +1,55 @@ +package cz.seznam.euphoria.beam.join; + +import cz.seznam.euphoria.beam.SingleValueCollector; +import cz.seznam.euphoria.core.client.functional.BinaryFunctor; +import cz.seznam.euphoria.core.client.util.Pair; +import org.apache.beam.sdk.transforms.join.CoGbkResult; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.TupleTag; + + +/** + * Left outer join implementation of {@link JoinFn}. + */ +public class LeftOuterJoinFn extends JoinFn { + + public LeftOuterJoinFn( + BinaryFunctor joiner, + TupleTag leftTag, + TupleTag rightTag) { + super(joiner, leftTag, rightTag); + } + + @Override + public void processElement(ProcessContext c) { + + KV element = c.element(); + CoGbkResult value = element.getValue(); + K key = element.getKey(); + + Iterable leftSideIter = value.getAll(leftTag); + Iterable rightSIdeIter = value.getAll(rightTag); + + SingleValueCollector outCollector = new SingleValueCollector<>(); + + for (LeftT leftValue : leftSideIter) { + if (rightSIdeIter.iterator().hasNext()) { + for (RightT rightValue : rightSIdeIter) { + joiner.apply(leftValue, rightValue, outCollector); + c.output(Pair.of(key, outCollector.get())); + } + } else { + joiner.apply(leftValue, null, outCollector); + c.output(Pair.of(key, outCollector.get())); + } + } + + + } + + @Override + public String getFnName() { + return "::left-outer-join"; + } + +} diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/LeftJoinFn.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/RightOuterJoinFn.java similarity index 55% rename from sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/LeftJoinFn.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/RightOuterJoinFn.java index e452b054e4545..354cf7b7fa7ab 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/LeftJoinFn.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/RightOuterJoinFn.java @@ -1,18 +1,18 @@ package cz.seznam.euphoria.beam.join; -/* + import cz.seznam.euphoria.beam.SingleValueCollector; import cz.seznam.euphoria.core.client.functional.BinaryFunctor; +import cz.seznam.euphoria.core.client.util.Pair; import org.apache.beam.sdk.transforms.join.CoGbkResult; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; -*/ /** - * Left Join implementation of {@link JoinFn}. + * Right outer join implementation of {@link JoinFn}. */ -public class LeftJoinFn /*extends JoinFn */{ -/* - protected LeftJoinFn( +public class RightOuterJoinFn extends JoinFn { + + public RightOuterJoinFn( BinaryFunctor joiner, TupleTag leftTag, TupleTag rightTag) { @@ -24,18 +24,29 @@ public void processElement(ProcessContext c) { KV element = c.element(); CoGbkResult value = element.getValue(); + K key = element.getKey(); Iterable leftSideIter = value.getAll(leftTag); Iterable rightSIdeIter = value.getAll(rightTag); SingleValueCollector outCollector = new SingleValueCollector<>(); + for (RightT rightValue : rightSIdeIter) { + if (leftSideIter.iterator().hasNext()) { + for (LeftT leftValue : leftSideIter) { + joiner.apply(leftValue, rightValue, outCollector); + c.output(Pair.of(key, outCollector.get())); + } + } else { + joiner.apply(null, rightValue, outCollector); + c.output(Pair.of(key, outCollector.get())); + } + } } @Override public String getFnName() { - return "::left-join"; + return "::right-outer-join"; } - */ } diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/JoinTest.java b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/JoinTest.java index d5ef67ddd39f9..148c946fc42d8 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/JoinTest.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/JoinTest.java @@ -6,9 +6,13 @@ import cz.seznam.euphoria.core.client.functional.BinaryFunctor; import cz.seznam.euphoria.core.client.io.ListDataSink; import cz.seznam.euphoria.core.client.io.ListDataSource; +import cz.seznam.euphoria.core.client.operator.FullJoin; import cz.seznam.euphoria.core.client.operator.Join; +import cz.seznam.euphoria.core.client.operator.LeftJoin; +import cz.seznam.euphoria.core.client.operator.RightJoin; import cz.seznam.euphoria.core.client.util.Pair; import cz.seznam.euphoria.testing.DatasetAssert; +import java.util.Optional; import org.junit.Test; /** @@ -59,24 +63,144 @@ public void simpleInnerJoinTest() { } -// public void someFutureTest() { -// -// ListDataSource> left = -// ListDataSource.unbounded( -// asList( -// Pair.of(1, "L v1"), Pair.of(1, "L v2"), Pair.of(1, "L v3"), -// Pair.of(2, "L v1"), Pair.of(2, "L v2"), -// Pair.of(3, "L v1") -// )); -// -// ListDataSource> right = -// ListDataSource.unbounded( -// asList( -// Pair.of(1, 1), Pair.of(1, 10), -// Pair.of(2, 20), -// Pair.of(3, 30), Pair.of(3, 300), Pair.of(3, 3000) -// )); -// } + @Test + public void simpleLeftJoinTest() { + final Flow flow = Flow.create(); + + ListDataSource> left = + ListDataSource.bounded( + asList( + Pair.of(1, "L v1"), Pair.of(1, "L v2"), + Pair.of(2, "L v1"), Pair.of(2, "L v2"), + Pair.of(3, "L v1") + )); + + ListDataSource> right = + ListDataSource.bounded( + asList( + Pair.of(1, 1), Pair.of(1, 10), + Pair.of(2, 20), + Pair.of(4, 40) + )); + + ListDataSink>> output = ListDataSink.get(); + + BinaryFunctor, Optional>, Pair> + joiner = (l, r, c) -> + c.collect(Pair.of(l.getSecond(), r.orElse(Pair.of(null, null)).getSecond())); + + LeftJoin.of(flow.createInput(left), flow.createInput(right)) + .by(Pair::getFirst, Pair::getFirst) + .using(joiner) + .output() + .persist(output); + + BeamExecutor executor = TestUtils.createExecutor(); + executor.execute(flow); + + DatasetAssert.unorderedEquals(output.getOutputs(), + Pair.of(1, Pair.of("L v1", 1)), Pair.of(1, Pair.of("L v1", 10)), + Pair.of(1, Pair.of("L v2", 1)), Pair.of(1, Pair.of("L v2", 10)), + + Pair.of(2, Pair.of("L v1", 20)), Pair.of(2, Pair.of("L v2", 20)), + + Pair.of(3, Pair.of("L v1", null)) + ); + + } + + @Test + public void simpleRightJoinTest() { + final Flow flow = Flow.create(); + + ListDataSource> left = + ListDataSource.bounded( + asList( + Pair.of(1, "L v1"), Pair.of(1, "L v2"), + Pair.of(2, "L v1"), Pair.of(2, "L v2"), + Pair.of(3, "L v1") + )); + + ListDataSource> right = + ListDataSource.bounded( + asList( + Pair.of(1, 1), Pair.of(1, 10), + Pair.of(2, 20), + Pair.of(4, 40) + )); + ListDataSink>> output = ListDataSink.get(); + + BinaryFunctor>, Pair, Pair> + joiner = (l, r, c) -> + c.collect(Pair.of(l.orElse(Pair.of(null, null)).getSecond(), r.getSecond())); + + RightJoin.of(flow.createInput(left), flow.createInput(right)) + .by(Pair::getFirst, Pair::getFirst) + .using(joiner) + .output() + .persist(output); + + BeamExecutor executor = TestUtils.createExecutor(); + executor.execute(flow); + + DatasetAssert.unorderedEquals(output.getOutputs(), + Pair.of(1, Pair.of("L v1", 1)), Pair.of(1, Pair.of("L v1", 10)), + Pair.of(1, Pair.of("L v2", 1)), Pair.of(1, Pair.of("L v2", 10)), + + Pair.of(2, Pair.of("L v1", 20)), Pair.of(2, Pair.of("L v2", 20)), + Pair.of(4, Pair.of(null, 40)) + ); + + } + + @Test + public void simpleFullJoinTest() { + final Flow flow = Flow.create(); + + ListDataSource> left = + ListDataSource.bounded( + asList( + Pair.of(1, "L v1"), Pair.of(1, "L v2"), + Pair.of(2, "L v1"), Pair.of(2, "L v2"), + Pair.of(3, "L v1") + )); + + ListDataSource> right = + ListDataSource.bounded( + asList( + Pair.of(1, 1), Pair.of(1, 10), + Pair.of(2, 20), + Pair.of(4, 40) + )); + + ListDataSink>> output = ListDataSink.get(); + + BinaryFunctor< + Optional>, Optional>, Pair> + joiner = (l, r, c) -> c.collect(Pair.of( + l.orElse(Pair.of(null, null)).getSecond(), r.orElse(Pair.of(null, null)).getSecond())); + + FullJoin.of(flow.createInput(left), flow.createInput(right)) + .by(Pair::getFirst, Pair::getFirst) + .using(joiner) + .output() + .persist(output); + + BeamExecutor executor = TestUtils.createExecutor(); + executor.execute(flow); + + DatasetAssert.unorderedEquals(output.getOutputs(), + Pair.of(1, Pair.of("L v1", 1)), Pair.of(1, Pair.of("L v1", 10)), + Pair.of(1, Pair.of("L v2", 1)), Pair.of(1, Pair.of("L v2", 10)), + + Pair.of(2, Pair.of("L v1", 20)), Pair.of(2, Pair.of("L v2", 20)), + + Pair.of(3, Pair.of("L v1", null)), + Pair.of(4, Pair.of(null, 40)) + ); + + } + } From 845ac4a783c912a2d32223d165f37670facd0f90 Mon Sep 17 00:00:00 2001 From: Vaclav Plajt Date: Thu, 17 May 2018 09:16:25 +0200 Subject: [PATCH 07/13] [BEAM-4294] BeamOperatorsSuite now includes tests of Join operator. Application of windowing in JoinTranslator fixed. --- .../seznam/euphoria/beam/JoinTranslator.java | 12 +- .../euphoria/beam/window/WindowingUtils.java | 3 +- .../cz/seznam/euphoria/beam/JoinTest.java | 2 +- .../beam/testkit/BeamOperatorsSuite.java | 3 +- .../euphoria-operator-testkit/build.gradle | 1 + .../euphoria/operator/test/JoinTest.java | 166 +++++++++++++----- 6 files changed, 140 insertions(+), 47 deletions(-) diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/JoinTranslator.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/JoinTranslator.java index 003eb60b6ba1b..77622c38a2d32 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/JoinTranslator.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/JoinTranslator.java @@ -45,7 +45,6 @@ public PCollection translate(Join operator, BeamExecutorContext context) { // get input data-sets transformed to Pcollections> List> inputs = context.getInputs(operator); - //TODO test left/right side indexes ! PCollection> leftKvInput = getKVInputCollection(inputs.get(0), operator.getLeftKeyExtractor(), keyCoder, new KryoCoder<>(), "::extract-keys-left"); @@ -54,6 +53,14 @@ public PCollection translate(Join operator, BeamExecutorContext context) { operator.getRightKeyExtractor(), keyCoder, new KryoCoder<>(), "::extract-keys-right"); + // and apply the same widowing on input Pcolections since the documentation states: + //'all of the PCollections you want to group must use the same + // windowing strategy and window sizing' + leftKvInput = WindowingUtils.applyWindowingIfSpecified( + operator, leftKvInput, context.getAllowedLateness(operator)); + rightKvInput = WindowingUtils.applyWindowingIfSpecified( + operator, rightKvInput, context.getAllowedLateness(operator)); + // GoGroupByKey collections TupleTag leftTag = new TupleTag<>(); TupleTag rightTag = new TupleTag<>(); @@ -63,9 +70,6 @@ public PCollection translate(Join operator, BeamExecutorContext context) { .and(rightTag, rightKvInput) .apply("::co-group-by-key", CoGroupByKey.create()); - coGrouped = WindowingUtils - .applyWindowingIfSpecified(operator, coGrouped, context.getAllowedLateness(operator)); - // Join JoinFn joinFn = chooseJoinFn(operator, leftTag, rightTag); diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/WindowingUtils.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/WindowingUtils.java index 39ee815bbee27..d546deb5f28d2 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/WindowingUtils.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/WindowingUtils.java @@ -26,7 +26,8 @@ PCollection applyWindowingIfSpecified( if (!(userSpecifiedWindowing instanceof BeamWindowing)) { throw new IllegalStateException(String.format( - "%s class only is supported to specify windowing.", BeamWindowing.class.getSimpleName())); + "Use of '%s' is only way supported to specify windowing.", + BeamWindowing.class.getSimpleName())); } @SuppressWarnings("unchecked") diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/JoinTest.java b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/JoinTest.java index 148c946fc42d8..7164f0d352255 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/JoinTest.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/JoinTest.java @@ -202,5 +202,5 @@ public void simpleFullJoinTest() { ); } - + } diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/testkit/BeamOperatorsSuite.java b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/testkit/BeamOperatorsSuite.java index 2a33cfae35a47..de76f39cebbc2 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/testkit/BeamOperatorsSuite.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/testkit/BeamOperatorsSuite.java @@ -18,6 +18,7 @@ package org.apache.beam.sdk.extensions.euphoria.beam.testkit; import org.apache.beam.sdk.extensions.euphoria.operator.test.FlatMapTest; +import org.apache.beam.sdk.extensions.euphoria.operator.test.JoinTest; import org.apache.beam.sdk.extensions.euphoria.operator.test.UnionTest; import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.ExecutorProvider; import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.ExecutorProviderRunner; @@ -36,7 +37,7 @@ // DistinctTest.class, // FilterTest.class, FlatMapTest.class, - // JoinTest.class, + JoinTest.class, // JoinWindowEnforcementTest.class, // MapElementsTest.class, // ReduceByKeyTest.class, diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/build.gradle b/sdks/java/extensions/euphoria/euphoria-operator-testkit/build.gradle index 3b0fce2be4e20..bd5abb4fad916 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/build.gradle +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/build.gradle @@ -23,5 +23,6 @@ description = "Apache Beam :: SDKs :: Java :: Extensions :: Euphoria Java 8 DSL" dependencies { compile project(':beam-sdks-java-extensions-euphoria-core') + compile project(':beam-sdks-java-extensions-euphoria-beam') compileOnly library.java.junit } diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/JoinTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/JoinTest.java index e05203fe0b668..033754bbc410d 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/JoinTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/JoinTest.java @@ -20,11 +20,14 @@ import static org.junit.Assert.assertEquals; import java.time.Duration; +import cz.seznam.euphoria.beam.window.BeamWindowing; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Optional; +import org.apache.beam.sdk.extensions.euphoria.beam.io.KryoCoder; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Session; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; @@ -48,9 +51,24 @@ import org.apache.beam.sdk.extensions.euphoria.operator.test.accumulators.SnapshotProvider; import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.AbstractOperatorTest; import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.Processing; +import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.transforms.windowing.AfterWatermark; +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.Sessions; +import org.apache.beam.sdk.transforms.windowing.WindowFn; +import org.apache.beam.sdk.transforms.windowing.WindowMappingFn; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowingStrategy.AccumulationMode; +import org.joda.time.Instant; +import org.junit.Ignore; import org.junit.Test; -/** Test operator {@code Join}. */ +/** + * Test operator {@code Join}. + */ @Processing(Processing.Type.ALL) public class JoinTest extends AbstractOperatorTest { @@ -271,7 +289,10 @@ protected Dataset> getOutput( (Optional l, Optional r, Collector c) -> { c.collect(l.orElse(null) + "+" + r.orElse(null)); }) - .windowBy(new EvenOddWindowing()) + .windowBy(BeamWindowing.of( + new EvenOddWindowFn(), + AfterWatermark.pastEndOfWindow(), + AccumulationMode.DISCARDING_FIRED_PANES)) .output(); } @@ -321,7 +342,10 @@ protected Dataset> getOutput( (Integer l, Optional r, Collector c) -> { c.collect(l + "+" + r.orElse(null)); }) - .windowBy(new EvenOddWindowing()) + .windowBy(BeamWindowing.of( + new EvenOddWindowFn(), + AfterWatermark.pastEndOfWindow(), + AccumulationMode.DISCARDING_FIRED_PANES)) .output(); } @@ -366,7 +390,10 @@ protected Dataset> getOutput( (Optional l, Long r, Collector c) -> { c.collect(l.orElse(null) + "+" + r); }) - .windowBy(new EvenOddWindowing()) + .windowBy(BeamWindowing.of( + new EvenOddWindowFn(), + AfterWatermark.pastEndOfWindow(), + AccumulationMode.DISCARDING_FIRED_PANES)) .output(); } @@ -403,7 +430,7 @@ public List> getUnorderedOutput() { public void joinOnSessionWindowingNoEarlyTriggering() { execute( new JoinTestCase< - Pair, Pair, Triple>() { + Pair, Pair, Pair>() { @Override protected List> getLeftInput() { @@ -416,37 +443,41 @@ protected List> getRightInput() { } @Override - protected Dataset> getOutput( + protected Dataset> getOutput( Dataset> left, Dataset> right) { + left = AssignEventTime.of(left).using(Pair::getSecond).output(); right = AssignEventTime.of(right).using(Pair::getSecond).output(); - Dataset>> joined = + + Dataset>> joined = Join.of(left, right) .by(p -> "", p -> "") .using( - (Pair l, - Pair r, - Collector> c) -> - c.collect( - Triple.of( - (TimeInterval) c.getWindow(), l.getFirst(), r.getFirst()))) - .windowBy(Session.of(Duration.ofMillis(10))) + (Pair l, Pair r, + Collector> c) -> + c.collect(Pair.of(l.getFirst(), r.getFirst()))) + .windowBy(BeamWindowing.of( + Sessions.withGapDuration(org.joda.time.Duration.millis(10)), + AfterWatermark.pastEndOfWindow(), + AccumulationMode.DISCARDING_FIRED_PANES)) .output(); return MapElements.of(joined).using(Pair::getSecond).output(); } @Override - public List> getUnorderedOutput() { - TimeInterval expectedWindow = new TimeInterval(1, 14); + public List> getUnorderedOutput() { return Arrays.asList( - Triple.of(expectedWindow, "fi", "ha"), - Triple.of(expectedWindow, "fi", "ho"), - Triple.of(expectedWindow, "fa", "ha"), - Triple.of(expectedWindow, "fa", "ho")); + Pair.of("fi", "ha"), + Pair.of("fi", "ho"), + Pair.of("fa", "ha"), + Pair.of("fa", "ho")); } }); } + @Ignore( + "This test is based on access to various objects through Environment which is " + + "unsupported feature. It may be possible to add this feature in future.") @Test public void testJoinAccumulators() { execute( @@ -466,8 +497,10 @@ protected List> getRightInput() { @Override protected Dataset> getOutput( Dataset> left, Dataset> right) { + left = AssignEventTime.of(left).using(Pair::getSecond).output(); right = AssignEventTime.of(right).using(Pair::getSecond).output(); + Dataset>> joined = Join.of(left, right) .by(p -> "", p -> "") @@ -480,7 +513,11 @@ protected Dataset> getOutput( c.getHistogram("hist-" + l.getFirst().charAt(1)).add(2345, 8); c.collect(Triple.of(window, l.getFirst(), r.getFirst())); }) - .windowBy(Time.of(Duration.ofMillis(3))) +// .windowBy(Time.of(Duration.ofMillis(3))) + .windowBy(BeamWindowing.of( + FixedWindows.of(org.joda.time.Duration.millis(3)), + AfterWatermark.pastEndOfWindow(), + AccumulationMode.DISCARDING_FIRED_PANES)) .output(); return MapElements.of(joined).using(Pair::getSecond).output(); } @@ -511,11 +548,9 @@ public void validateAccumulators(SnapshotProvider snapshots) { /** * Base for join test cases. - * @param - * @param - * @param */ public abstract static class JoinTestCase implements TestCase { + @Override public Dataset getOutput(Flow flow, boolean bounded) { Dataset left = flow.createInput(ListDataSource.of(bounded, getLeftInput())); @@ -530,36 +565,87 @@ public Dataset getOutput(Flow flow, boolean bounded) { protected abstract List getRightInput(); } - /** Stable windowing for test purposes. */ - static class EvenOddWindowing implements Windowing, IntWindow> { + + /** + * Elements with even numeric values are are assigned to one 'even' window. All others are + * assigned to window named 'win: #', where '#' is value of assigned element. + */ + private static class EvenOddWindowFn extends WindowFn, BoundedWindow> { + + private static final NamedGlobalWindow EVEN_WIN = new NamedGlobalWindow("even"); @Override - public Iterable assignWindowsToElement( - WindowedElement> input) { - int element; - Either unwrapped = input.getElement(); - if (unwrapped.isLeft()) { - element = unwrapped.left(); + @SuppressFBWarnings(value = "RCN_REDUNDANT_NULLCHECK_OF_NONNULL_VALUE") + public Collection assignWindows(AssignContext c) throws Exception { + KV element = c.element(); + + Number value = element.getValue(); + + if (value == null) { + return Collections.singleton(EVEN_WIN); + } + + NamedGlobalWindow win; + if (value.longValue() % 2 == 0) { + win = EVEN_WIN; } else { - element = (int) (long) unwrapped.right(); + win = new NamedGlobalWindow("win: " + value.longValue()); } - final int label = element % 2 == 0 ? 0 : element; - return Collections.singleton(new IntWindow(label)); + + return Collections.singleton(win); } @Override - public Trigger getTrigger() { - return NoopTrigger.get(); + public void mergeWindows(MergeContext c) throws Exception { + // no merging } @Override - public boolean equals(Object obj) { - return obj instanceof EvenOddWindowing; + public boolean isCompatible(WindowFn other) { + return other instanceof EvenOddWindowFn; + } + + @Override + public Coder windowCoder() { + return new KryoCoder<>(); + } + + @Override + @Nullable + public WindowMappingFn getDefaultWindowMappingFn() { + return null; + } + + @Override + public boolean isNonMerging() { + return true; + } + } + + private static class NamedGlobalWindow extends BoundedWindow { + + private String name; + + public NamedGlobalWindow(String name) { + this.name = name; + } + + @Override + public Instant maxTimestamp() { + return GlobalWindow.INSTANCE.maxTimestamp(); + } + + @Override + public boolean equals(Object other) { + if (other instanceof NamedGlobalWindow) { + return name.equals(((NamedGlobalWindow) other).name); + } + return false; } @Override public int hashCode() { - return 0; + return name.hashCode(); } } } From f7956d0523fd79c60de5caa65825d3bcfc1bd104 Mon Sep 17 00:00:00 2001 From: Vaclav Plajt Date: Thu, 17 May 2018 16:22:52 +0200 Subject: [PATCH 08/13] [BEAM-4360] ReduceStateByKey tests adaptation to beam. - Work in progress. --- .../euphoria/beam/ReduceByKeyTranslator.java | 4 + .../beam/testkit/BeamOperatorsSuite.java | 3 +- .../operator/test/ReduceByKeyTest.java | 211 +++++++++++++----- 3 files changed, 165 insertions(+), 53 deletions(-) diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTranslator.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTranslator.java index c22492e8591d8..bf3a30ee391f8 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTranslator.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTranslator.java @@ -50,6 +50,10 @@ class ReduceByKeyTranslator implements OperatorTranslator { private static > PCollection> doTranslate(ReduceByKey operator, BeamExecutorContext context) { + if (operator.getValueComparator() != null) { //TODO Could we even do values sorting ? + throw new UnsupportedOperationException("Values sorting is not supported."); + } + final UnaryFunction keyExtractor = operator.getKeyExtractor(); final UnaryFunction valueExtractor = operator.getValueExtractor(); final ReduceFunctor reducer = operator.getReducer(); diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/testkit/BeamOperatorsSuite.java b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/testkit/BeamOperatorsSuite.java index de76f39cebbc2..bc1aa26571914 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/testkit/BeamOperatorsSuite.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/testkit/BeamOperatorsSuite.java @@ -19,6 +19,7 @@ import org.apache.beam.sdk.extensions.euphoria.operator.test.FlatMapTest; import org.apache.beam.sdk.extensions.euphoria.operator.test.JoinTest; +import org.apache.beam.sdk.extensions.euphoria.operator.test.ReduceByKeyTest; import org.apache.beam.sdk.extensions.euphoria.operator.test.UnionTest; import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.ExecutorProvider; import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.ExecutorProviderRunner; @@ -40,7 +41,7 @@ JoinTest.class, // JoinWindowEnforcementTest.class, // MapElementsTest.class, - // ReduceByKeyTest.class, + ReduceByKeyTest.class, // ReduceStateByKeyTest.class, // SumByKeyTest.class, // TopPerKeyTest.class, diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/ReduceByKeyTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/ReduceByKeyTest.java index e23ceb6d25780..0f63b74b851c0 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/ReduceByKeyTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/ReduceByKeyTest.java @@ -24,6 +24,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Sets; import java.time.Duration; +import cz.seznam.euphoria.beam.window.BeamWindowing; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -35,6 +36,7 @@ import java.util.Set; import java.util.stream.Collectors; import java.util.stream.Stream; +import org.apache.beam.sdk.extensions.euphoria.beam.io.KryoCoder; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Count; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.GlobalWindowing; @@ -68,20 +70,28 @@ import org.apache.beam.sdk.extensions.euphoria.operator.test.accumulators.SnapshotProvider; import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.AbstractOperatorTest; import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.Processing; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.transforms.windowing.AfterWatermark; +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.GlobalWindows; +import org.apache.beam.sdk.transforms.windowing.WindowFn; +import org.apache.beam.sdk.transforms.windowing.WindowMappingFn; +import org.apache.beam.sdk.values.WindowingStrategy.AccumulationMode; +import org.joda.time.Instant; +import org.junit.Ignore; import org.junit.Test; /** Test operator {@code ReduceByKey}. */ @Processing(Processing.Type.ALL) public class ReduceByKeyTest extends AbstractOperatorTest { - /** Validates the output type upon a `.reduceBy` operation on windows of size one. */ + /** Validates the output type upon a `.reduceBy` operation on global window. */ @Test public void testReductionType0() { execute( - new AbstractTestCase>>( - /* don't parallelize this test, because it doesn't work - * well with count windows */ - 1) { + new AbstractTestCase>>() { @Override protected List getInput() { return Arrays.asList(1, 2, 3, 4, 5, 6, 7, 9); @@ -93,7 +103,10 @@ protected Dataset>> getOutput(Dataset input) .keyBy(e -> e % 2) .valueBy(e -> e) .reduceBy(s -> s.collect(Collectors.toSet())) - .windowBy(Count.of(3)) + .windowBy(BeamWindowing.of( + new GlobalWindows(), + AfterWatermark.pastEndOfWindow(), + AccumulationMode.DISCARDING_FIRED_PANES)) .output(); } @@ -101,20 +114,16 @@ protected Dataset>> getOutput(Dataset input) public List>> getUnorderedOutput() { return Arrays.asList( Pair.of(0, Sets.newHashSet(2, 4, 6)), - Pair.of(1, Sets.newHashSet(1, 3, 5)), - Pair.of(1, Sets.newHashSet(7, 9))); + Pair.of(1, Sets.newHashSet(1, 3, 5, 7, 9))); } }); } - /** Validates the output type upon a `.reduceBy` operation on windows of size one. */ + /** Validates the output type upon a `.reduceBy` operation on global window. */ @Test public void testReductionType0_outputValues() { execute( - new AbstractTestCase>( - /* don't parallelize this test, because it doesn't work - * well with count windows */ - 1) { + new AbstractTestCase>() { @Override protected List getInput() { return Arrays.asList(1, 2, 3, 4, 5, 6, 7, 9); @@ -126,19 +135,23 @@ protected Dataset> getOutput(Dataset input) { .keyBy(e -> e % 2) .valueBy(e -> e) .reduceBy(s -> s.collect(Collectors.toSet())) - .windowBy(Count.of(3)) + .windowBy(BeamWindowing.of( + new GlobalWindows(), + AfterWatermark.pastEndOfWindow(), + AccumulationMode.DISCARDING_FIRED_PANES)) .outputValues(); } @Override public List> getUnorderedOutput() { return Arrays.asList( - Sets.newHashSet(2, 4, 6), Sets.newHashSet(1, 3, 5), Sets.newHashSet(7, 9)); + Sets.newHashSet(2, 4, 6), Sets.newHashSet(1, 3, 5, 7, 9)); } }); } - /** Validates the output type upon a `.reduceBy` operation on windows of size one. */ + /** Validates the output type upon a `.reduceBy` operation on global window. */ + @Ignore("Sorting of values is not supported yet.") @Test public void testReductionType0WithSortedValues() { execute( @@ -174,7 +187,10 @@ protected Dataset>>> getOutput(Dataset } return cmp; }) - .windowBy(GlobalWindowing.get()) + .windowBy(BeamWindowing.of( + new GlobalWindows(), + AfterWatermark.pastEndOfWindow(), + AccumulationMode.DISCARDING_FIRED_PANES)) .output(); } @@ -213,7 +229,11 @@ protected Dataset> getOutput(Dataset input) { return ReduceByKey.of(input) .keyBy(e -> e % 2) .reduceBy(Fold.whileEmittingEach(0, (a, b) -> a + b)) - .windowBy(Count.of(3)) +// .windowBy(Count.of(3)) + .windowBy(BeamWindowing.of( + new GlobalWindows(), + AfterWatermark.pastEndOfWindow(), + AccumulationMode.DISCARDING_FIRED_PANES)) .output(); } @@ -234,12 +254,13 @@ public void validate(List> output) { assertEquals(Arrays.asList(2, 6, 12), byKey.get(0)); assertNotNull(byKey.get(1)); - assertEquals(Sets.newHashSet(1, 4, 9, 7, 16), new HashSet<>(byKey.get(1))); + assertEquals(Sets.newHashSet(1, 4, 9, 16, 25), new HashSet<>(byKey.get(1))); } @Override public List> getUnorderedOutput() { - return Arrays.asList(Pair.of(0, 12), Pair.of(1, 9), Pair.of(1, 16)); +// return Arrays.asList(Pair.of(0, 12), Pair.of(1, 9), Pair.of(1, 16)); + return Arrays.asList(Pair.of(0, 12), Pair.of(1, 25)); } }); } @@ -256,7 +277,11 @@ protected Dataset> getOutput(Dataset> in .keyBy(Pair::getFirst) .valueBy(e -> 1L) .combineBy(Sums.ofLongs()) - .windowBy(Time.of(Duration.ofSeconds(1))) +// .windowBy(Time.of(Duration.ofSeconds(1))) + .windowBy(BeamWindowing.of( + FixedWindows.of(org.joda.time.Duration.standardSeconds(1)), + AfterWatermark.pastEndOfWindow(), + AccumulationMode.DISCARDING_FIRED_PANES)) .output(); } @@ -318,7 +343,11 @@ protected Dataset> getOutput(Dataset input) { .keyBy(e -> e % 3) .valueBy(e -> 1L) .combineBy(Sums.ofLongs()) - .windowBy(new TestWindowing()) +// .windowBy(new TestWindowing()) + .windowBy(BeamWindowing.of( + new TestWindowFn(), + AfterWatermark.pastEndOfWindow(), + AccumulationMode.DISCARDING_FIRED_PANES)) .output(); } @@ -398,6 +427,7 @@ protected Dataset> getOutput(Dataset input) { }); } + @Ignore("Sorting of values is not supported yet.") @Processing(Processing.Type.BOUNDED) @Test public void testReduceSorted() { @@ -485,7 +515,7 @@ public List> getUnorderedOutput() { } // ---------------------------------------------------------------------------- - + @Ignore("Test depends on yet unsupported functionality (access to window from Collector). ") @Test public void testSessionWindowing() { execute( @@ -514,7 +544,10 @@ protected Dataset>> getOutput( .keyBy(e -> e.getFirst().charAt(0) - '0') .valueBy(Pair::getFirst) .reduceBy(s -> s.collect(Collectors.toSet())) - .windowBy(Session.of(Duration.ofMillis(5))) + .windowBy(BeamWindowing.of( + FixedWindows.of(org.joda.time.Duration.millis(5)), + AfterWatermark.pastEndOfWindow(), + AccumulationMode.DISCARDING_FIRED_PANES)) .output(); return FlatMap.of(reduced) @@ -545,46 +578,44 @@ public List>> getUnorderedOutput() { }); } + + @Ignore("Test depends on unsupported ReduceStateByKey operator.") @Test public void testElementTimestamp() { - class AssertingWindowing implements Windowing { + + class AssertingWindowFn extends WindowFn{ + @Override - public Iterable assignWindowsToElement(WindowedElement el) { + public Collection assignWindows(AssignContext c) throws Exception { + long timestamp = c.timestamp().getMillis(); + // ~ we expect the 'element time' to be the end of the window which produced the // element in the preceding upstream (stateful and windowed) operator assertTrue( - "Invalid timestamp " + el.getTimestamp(), - el.getTimestamp() == 15_000L - 1 || el.getTimestamp() == 25_000L - 1); - return Collections.singleton(new TimeInterval(0, Long.MAX_VALUE)); + "Invalid timestamp " + timestamp, + timestamp == 15_000L - 1 || timestamp == 25_000L - 1); + + return Collections.singleton(GlobalWindow.INSTANCE); } - @SuppressWarnings("unchecked") @Override - public Trigger getTrigger() { - return new CountTrigger(1) { - @Override - public boolean isStateful() { - return false; - } + public void mergeWindows(MergeContext c) throws Exception { - @Override - public TriggerResult onElement(long time, Window window, TriggerContext ctx) { - // ~ we expect the 'time' to be the end of the window which produced the - // element in the preceding upstream (stateful and windowed) operator - assertTrue("Invalid timestamp " + time, time == 15_000L - 1 || time == 25_000L - 1); - return super.onElement(time, window, ctx); - } - }; } @Override - public boolean equals(Object obj) { - return obj instanceof AssertingWindowing; + public boolean isCompatible(WindowFn other) { + return other instanceof GlobalWindows; + } + + @Override + public Coder windowCoder() { + return new KryoCoder<>(); //TODO do we need this? } @Override - public int hashCode() { - return 0; + public WindowMappingFn getDefaultWindowMappingFn() { + return null; } } @@ -612,7 +643,11 @@ protected Dataset getOutput(Dataset> input) { .keyBy(e -> "", TypeHint.ofString()) .valueBy(Pair::getFirst, TypeHint.ofInt()) .combineBy(Sums.ofInts(), TypeHint.ofInt()) - .windowBy(Time.of(Duration.ofSeconds(5))) +// .windowBy(Time.of(Duration.ofSeconds(5))) + .windowBy(BeamWindowing.of( + FixedWindows.of(org.joda.time.Duration.standardSeconds(5)), + AfterWatermark.pastEndOfWindow(), + AccumulationMode.DISCARDING_FIRED_PANES)) .output(); // ~ now use a custom windowing with a trigger which does // the assertions subject to this test (use RSBK which has to @@ -623,7 +658,10 @@ protected Dataset getOutput(Dataset> input) { .valueBy(Pair::getSecond) .stateFactory(SumState::new) .mergeStatesBy(SumState::combine) - .windowBy(new AssertingWindowing<>()) + .windowBy(BeamWindowing.of( + new AssertingWindowFn<>(), + AfterWatermark.pastEndOfWindow(), + AccumulationMode.DISCARDING_FIRED_PANES)) .output(); return FlatMap.of(output) .using( @@ -651,7 +689,11 @@ protected Dataset> getOutput(Dataset> input) { .keyBy(Pair::getFirst) .valueBy(e -> 1L) .combineBy(Sums.ofLongs()) - .windowBy(Time.of(Duration.ofSeconds(1))) +// .windowBy(Time.of(Duration.ofSeconds(1))) + .windowBy(BeamWindowing.of( + FixedWindows.of(org.joda.time.Duration.standardSeconds(1)), + AfterWatermark.pastEndOfWindow(), + AccumulationMode.DISCARDING_FIRED_PANES)) .output(); } @@ -703,7 +745,11 @@ protected Dataset> getOutput(Dataset input) { } ctx.collect(a + b); })) - .windowBy(GlobalWindowing.get()) +// .windowBy(GlobalWindowing.get()) + .windowBy(BeamWindowing.of( + new GlobalWindows(), + AfterWatermark.pastEndOfWindow(), + AccumulationMode.DISCARDING_FIRED_PANES)) .output(); } @@ -745,8 +791,69 @@ public int hashCode() { } } + private static class TestWindowFn extends WindowFn{ + + @Override + public Collection assignWindows(AssignContext c) throws Exception { + Number element = c.element(); + return Collections.singleton(new CountWindow(element.longValue() / 4 )); //TODO why /4 ? + } + + @Override + public void mergeWindows(MergeContext c) throws Exception { + + } + + @Override + public boolean isNonMerging() { + return true; + } + + @Override + public boolean isCompatible(WindowFn other) { + return false; + } + + @Override + public Coder windowCoder() { + return new KryoCoder<>(); + } + + @Override + public WindowMappingFn getDefaultWindowMappingFn() { + return null; + } + } + // ~ ------------------------------------------------------------------------------ + private static class CountWindow extends BoundedWindow { + + private long value; + + public CountWindow(long value) { + this.value = value; + } + + @Override + public Instant maxTimestamp() { + return GlobalWindow.INSTANCE.maxTimestamp(); + } + + @Override + public boolean equals(Object other) { + if (other instanceof CountWindow) { + return value == (((CountWindow) other).value); + } + return false; + } + + @Override + public int hashCode() { + return Long.hashCode(value); + } + } + /** * Every instance is unique: this allows us to exercise merging. */ From 4d358fd0a466483b85d63d534a011de31d61418d Mon Sep 17 00:00:00 2001 From: Vaclav Plajt Date: Fri, 18 May 2018 15:11:27 +0200 Subject: [PATCH 09/13] [BEAM-4360] Most tests are passing. Some are still ignored mostly due to yet unsupported features. --- .../euphoria/euphoria-beam/build.gradle | 4 +- .../operator/test/ReduceByKeyTest.java | 151 +++++++++++++++--- 2 files changed, 131 insertions(+), 24 deletions(-) diff --git a/sdks/java/extensions/euphoria/euphoria-beam/build.gradle b/sdks/java/extensions/euphoria/euphoria-beam/build.gradle index df0a31fef894c..65f8a365cdb3b 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/build.gradle +++ b/sdks/java/extensions/euphoria/euphoria-beam/build.gradle @@ -31,11 +31,9 @@ dependencies { shadow 'com.google.code.findbugs:annotations:3.0.1' testCompile project(':beam-sdks-java-extensions-euphoria-operator-testkit') testCompile project(':beam-sdks-java-extensions-euphoria-testing') - //testCompile project(':beam-runners-direct-java') + testCompile project(':beam-runners-direct-java') testCompile library.java.slf4j_api testCompile library.java.hamcrest_core - testCompile "org.apache.beam:beam-runners-direct-java:2.4.0" - } test.testLogging.showStandardStreams = true diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/ReduceByKeyTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/ReduceByKeyTest.java index 0f63b74b851c0..95040a12e4bae 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/ReduceByKeyTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/ReduceByKeyTest.java @@ -34,6 +34,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.beam.sdk.extensions.euphoria.beam.io.KryoCoder; @@ -70,6 +71,7 @@ import org.apache.beam.sdk.extensions.euphoria.operator.test.accumulators.SnapshotProvider; import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.AbstractOperatorTest; import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.Processing; +import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.windowing.AfterWatermark; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; @@ -83,11 +85,15 @@ import org.junit.Ignore; import org.junit.Test; -/** Test operator {@code ReduceByKey}. */ +/** + * Test operator {@code ReduceByKey}. + */ @Processing(Processing.Type.ALL) public class ReduceByKeyTest extends AbstractOperatorTest { - /** Validates the output type upon a `.reduceBy` operation on global window. */ + /** + * Validates the output type upon a `.reduceBy` operation on global window. + */ @Test public void testReductionType0() { execute( @@ -119,7 +125,9 @@ public List>> getUnorderedOutput() { }); } - /** Validates the output type upon a `.reduceBy` operation on global window. */ + /** + * Validates the output type upon a `.reduceBy` operation on global window. + */ @Test public void testReductionType0_outputValues() { execute( @@ -150,7 +158,9 @@ public List> getUnorderedOutput() { }); } - /** Validates the output type upon a `.reduceBy` operation on global window. */ + /** + * Validates the output type upon a `.reduceBy` operation on global window. + */ @Ignore("Sorting of values is not supported yet.") @Test public void testReductionType0WithSortedValues() { @@ -210,7 +220,9 @@ public void validate(List>>> outputs) }); } - /** Validates the output type upon a `.reduceBy` operation on windows of size one. */ + /** + * Validates the output type upon a `.reduceBy` operation on windows of size one. + */ @Test public void testReductionType0MultiValues() { execute( @@ -470,6 +482,7 @@ protected Dataset>> getOutput(Dataset }); } + @Ignore("Test adaption to Beam windowing failed so far.") @Test public void testMergingAndTriggering() { execute( @@ -479,14 +492,14 @@ public void testMergingAndTriggering() { protected List> getInput() { return Arrays.asList( Pair.of("a", 20L), - Pair.of("c", 3000L), + Pair.of("c", 3_000L), Pair.of("b", 10L), Pair.of("b", 100L), - Pair.of("a", 4000L), + Pair.of("a", 4_000L), Pair.of("c", 300L), - Pair.of("b", 1000L), - Pair.of("b", 50000L), - Pair.of("a", 100000L), + Pair.of("b", 1_000L), + Pair.of("b", 50_000L), + Pair.of("a", 100_000L), Pair.of("a", 800L), Pair.of("a", 80L)); } @@ -497,7 +510,11 @@ protected Dataset> getOutput(Dataset> inpu .keyBy(Pair::getFirst) .valueBy(Pair::getSecond) .combineBy(Sums.ofLongs()) - .windowBy(new CWindowing<>(3)) +// .windowBy(new CWindowing<>(3)) + .windowBy(BeamWindowing.of( + new MergingByBucketSizeWindowFn<>(3), + AfterWatermark.pastEndOfWindow(), + AccumulationMode.DISCARDING_FIRED_PANES)) .output(); } @@ -506,10 +523,10 @@ protected Dataset> getOutput(Dataset> inpu public List> getUnorderedOutput() { return Arrays.asList( Pair.of("a", 880L), - Pair.of("a", 104020L), - Pair.of("b", 1110L), - Pair.of("b", 50000L), - Pair.of("c", 3300L)); + Pair.of("a", 104_020L), + Pair.of("b", 1_110L), + Pair.of("b", 50_000L), + Pair.of("c", 3_300L)); } }); } @@ -553,7 +570,7 @@ protected Dataset>> getOutput( return FlatMap.of(reduced) .using( (UnaryFunctor< - Pair>, Triple>>) + Pair>, Triple>>) (elem, context) -> context.collect( Triple.of( @@ -583,7 +600,7 @@ public List>> getUnorderedOutput() { @Test public void testElementTimestamp() { - class AssertingWindowFn extends WindowFn{ + class AssertingWindowFn extends WindowFn { @Override public Collection assignWindows(AssignContext c) throws Exception { @@ -610,10 +627,11 @@ public boolean isCompatible(WindowFn other) { @Override public Coder windowCoder() { - return new KryoCoder<>(); //TODO do we need this? + return new KryoCoder<>(); } @Override + @Nullable public WindowMappingFn getDefaultWindowMappingFn() { return null; } @@ -791,12 +809,12 @@ public int hashCode() { } } - private static class TestWindowFn extends WindowFn{ + private static class TestWindowFn extends WindowFn { @Override public Collection assignWindows(AssignContext c) throws Exception { Number element = c.element(); - return Collections.singleton(new CountWindow(element.longValue() / 4 )); //TODO why /4 ? + return Collections.singleton(new CountWindow(element.longValue() / 4)); } @Override @@ -820,6 +838,7 @@ public Coder windowCoder() { } @Override + @Nullable public WindowMappingFn getDefaultWindowMappingFn() { return null; } @@ -854,6 +873,37 @@ public int hashCode() { } } + private static class UniqueWindow extends BoundedWindow { + + private static final AtomicInteger idCounter = new AtomicInteger(); + private final int id; + + public UniqueWindow() { + this.id = idCounter.getAndIncrement(); + } + + @Override + public Instant maxTimestamp() { + return GlobalWindow.INSTANCE.maxTimestamp(); + } + + @Override + public int hashCode() { + return Integer.hashCode(id); + } + + @Override + public boolean equals(Object obj) { + return obj instanceof UniqueWindow + && this.id == ((UniqueWindow) obj).id; + } + + @Override + public String toString() { + return "UniqueWindow{id=" + id + "}"; + } + } + /** * Every instance is unique: this allows us to exercise merging. */ @@ -898,6 +948,7 @@ public String toString() { // count windowing; firing based on window.bucket (size of the window) static final class CWindowTrigger implements Trigger { + private final ValueStorageDescriptor countDesc = ValueStorageDescriptor.of("count", Long.class, 0L, (x, y) -> x + y); @@ -927,7 +978,62 @@ public void onMerge(CWindow w, TriggerContext.TriggerMergeContext ctx) { } } + private static class MergingByBucketSizeWindowFn extends WindowFn { + + private final int bucketSize; + + private MergingByBucketSizeWindowFn(int bucketSize) { + this.bucketSize = bucketSize; + } + + @Override + public Collection assignWindows(AssignContext c) throws Exception { + return Collections.singleton(new UniqueWindow()); + } + + @Override + public void mergeWindows(MergeContext c) throws Exception { + +// merge windows up to bucket size + Collection windows = c.windows(); + List merges = new ArrayList<>(); + for (UniqueWindow w : windows) { + + merges.add(w); + + if (merges.size() == bucketSize) { // time to merge + c.merge(merges, w); + merges.clear(); + } + + } + + if (merges.size() > 1) { + c.merge(merges, merges.get(merges.size() - 1)); + } + + } + + @Override + public boolean isCompatible(WindowFn other) { + return other instanceof MergingByBucketSizeWindowFn + && this.bucketSize == ((MergingByBucketSizeWindowFn) other).bucketSize; + } + + @Override + public Coder windowCoder() { + return new KryoCoder<>(); + } + + @Override + @Nullable + public WindowMappingFn getDefaultWindowMappingFn() { + return null; + } + } + static final class CWindowing implements MergingWindowing { + private final int size; CWindowing(int size) { @@ -976,6 +1082,7 @@ public int hashCode() { } static class SumState implements State { + private final ValueStorage sum; SumState(StateContext context, Collector collector) { @@ -1007,7 +1114,9 @@ public void close() { } } - /** String with invalid hash code implementation returning constant. */ + /** + * String with invalid hash code implementation returning constant. + */ public static class Word { private final String str; From f3558b0c7d9b12f764e3df0319da6288ee92c383 Mon Sep 17 00:00:00 2001 From: Vaclav Plajt Date: Mon, 21 May 2018 08:46:45 +0200 Subject: [PATCH 10/13] [BEAM-4294] [BEAM-4360] Join translation and ReduceByKey test suite where moved to org.apache.beam.* package. Small imports fixes after rebase to seznam/beam eff3ffd5d224c6628606b355e3a5b6d8bc40150b. --- .../euphoria/beam/FlowTranslator.java | 1 - .../euphoria/beam/JoinTranslator.java | 30 +++++++++---------- .../euphoria/beam/ReduceByKeyTranslator.java | 4 +-- .../euphoria/beam/common/InputToKvDoFn.java | 4 +-- .../euphoria/beam/common/package-info.java | 2 +- .../euphoria/beam/join/FullJoinFn.java | 8 ++--- .../euphoria/beam/join/InnerJoinFn.java | 8 ++--- .../euphoria/beam/join/JoinFn.java | 6 ++-- .../euphoria/beam/join/LeftOuterJoinFn.java | 8 ++--- .../euphoria/beam/join/RightOuterJoinFn.java | 8 ++--- .../euphoria/beam/join/package-info.java | 2 +- .../euphoria/beam/window/BeamWindowing.java | 2 +- .../beam/window/UnsupportedWindow.java | 2 +- .../euphoria/beam/window/WindowingUtils.java | 8 ++--- .../extensions}/euphoria/beam/JoinTest.java | 22 +++++++------- .../euphoria/beam/ReduceByKeyTest.java | 6 +--- .../extensions}/euphoria/beam/TestUtils.java | 2 +- .../euphoria/operator/test/JoinTest.java | 11 ++----- .../operator/test/ReduceByKeyTest.java | 7 +---- 19 files changed, 61 insertions(+), 80 deletions(-) rename sdks/java/extensions/euphoria/euphoria-beam/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/JoinTranslator.java (80%) rename sdks/java/extensions/euphoria/euphoria-beam/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/common/InputToKvDoFn.java (81%) rename sdks/java/extensions/euphoria/euphoria-beam/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/common/package-info.java (93%) rename sdks/java/extensions/euphoria/euphoria-beam/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/join/FullJoinFn.java (85%) rename sdks/java/extensions/euphoria/euphoria-beam/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/join/InnerJoinFn.java (80%) rename sdks/java/extensions/euphoria/euphoria-beam/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/join/JoinFn.java (82%) rename sdks/java/extensions/euphoria/euphoria-beam/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/join/LeftOuterJoinFn.java (82%) rename sdks/java/extensions/euphoria/euphoria-beam/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/join/RightOuterJoinFn.java (82%) rename sdks/java/extensions/euphoria/euphoria-beam/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/join/package-info.java (93%) rename sdks/java/extensions/euphoria/euphoria-beam/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/window/BeamWindowing.java (97%) rename sdks/java/extensions/euphoria/euphoria-beam/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/window/UnsupportedWindow.java (96%) rename sdks/java/extensions/euphoria/euphoria-beam/src/main/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/window/WindowingUtils.java (85%) rename sdks/java/extensions/euphoria/euphoria-beam/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/JoinTest.java (88%) rename sdks/java/extensions/euphoria/euphoria-beam/src/test/java/{cz/seznam => org/apache/beam/sdk/extensions}/euphoria/beam/TestUtils.java (90%) diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/FlowTranslator.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/FlowTranslator.java index e8d9af270c62e..25891bb3afb6b 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/FlowTranslator.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/FlowTranslator.java @@ -17,7 +17,6 @@ */ package org.apache.beam.sdk.extensions.euphoria.beam; -import cz.seznam.euphoria.beam.JoinTranslator; import java.util.IdentityHashMap; import java.util.Map; import org.apache.beam.sdk.Pipeline; diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/JoinTranslator.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/JoinTranslator.java similarity index 80% rename from sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/JoinTranslator.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/JoinTranslator.java index 77622c38a2d32..888aaf0996c77 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/JoinTranslator.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/JoinTranslator.java @@ -1,21 +1,21 @@ -package cz.seznam.euphoria.beam; - -import cz.seznam.euphoria.beam.common.InputToKvDoFn; -import cz.seznam.euphoria.beam.io.KryoCoder; -import cz.seznam.euphoria.beam.join.FullJoinFn; -import cz.seznam.euphoria.beam.join.InnerJoinFn; -import cz.seznam.euphoria.beam.join.JoinFn; -import cz.seznam.euphoria.beam.join.LeftOuterJoinFn; -import cz.seznam.euphoria.beam.join.RightOuterJoinFn; -import cz.seznam.euphoria.beam.window.WindowingUtils; -import cz.seznam.euphoria.core.client.dataset.windowing.Window; -import cz.seznam.euphoria.core.client.functional.BinaryFunctor; -import cz.seznam.euphoria.core.client.functional.UnaryFunction; -import cz.seznam.euphoria.core.client.operator.Join; -import cz.seznam.euphoria.core.client.util.Pair; +package org.apache.beam.sdk.extensions.euphoria.beam; + +import org.apache.beam.sdk.extensions.euphoria.beam.common.InputToKvDoFn; +import org.apache.beam.sdk.extensions.euphoria.beam.join.FullJoinFn; +import org.apache.beam.sdk.extensions.euphoria.beam.join.InnerJoinFn; +import org.apache.beam.sdk.extensions.euphoria.beam.join.JoinFn; +import org.apache.beam.sdk.extensions.euphoria.beam.join.LeftOuterJoinFn; +import org.apache.beam.sdk.extensions.euphoria.beam.join.RightOuterJoinFn; +import org.apache.beam.sdk.extensions.euphoria.beam.window.WindowingUtils; import java.util.List; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.extensions.euphoria.beam.io.KryoCoder; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.BinaryFunctor; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Join; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.join.CoGbkResult; import org.apache.beam.sdk.transforms.join.CoGroupByKey; diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTranslator.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTranslator.java index bf3a30ee391f8..5e74ebff67d97 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTranslator.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTranslator.java @@ -18,15 +18,13 @@ package org.apache.beam.sdk.extensions.euphoria.beam; -import cz.seznam.euphoria.beam.window.WindowingUtils; +import org.apache.beam.sdk.extensions.euphoria.beam.window.WindowingUtils; import java.util.stream.StreamSupport; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.coders.KvCoder; -import org.apache.beam.sdk.extensions.euphoria.beam.window.BeamWindowFn; import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.AccumulatorProvider; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; -import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Windowing; import org.apache.beam.sdk.extensions.euphoria.core.client.functional.ReduceFunctor; import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction; import org.apache.beam.sdk.extensions.euphoria.core.client.operator.ReduceByKey; diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/common/InputToKvDoFn.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/common/InputToKvDoFn.java similarity index 81% rename from sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/common/InputToKvDoFn.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/common/InputToKvDoFn.java index dbac09e738fec..83670856352f9 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/common/InputToKvDoFn.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/common/InputToKvDoFn.java @@ -1,6 +1,6 @@ -package cz.seznam.euphoria.beam.common; +package org.apache.beam.sdk.extensions.euphoria.beam.common; -import cz.seznam.euphoria.core.client.functional.UnaryFunction; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.values.KV; diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/common/package-info.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/common/package-info.java similarity index 93% rename from sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/common/package-info.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/common/package-info.java index c238720a5c4d9..2de3d603025a3 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/common/package-info.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/common/package-info.java @@ -19,4 +19,4 @@ /** * A set of commonly used classes enabling some code reuse. */ -package cz.seznam.euphoria.beam.common; +package org.apache.beam.sdk.extensions.euphoria.beam.common; diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/FullJoinFn.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/join/FullJoinFn.java similarity index 85% rename from sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/FullJoinFn.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/join/FullJoinFn.java index b04faa971d93f..8333c41d18aef 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/FullJoinFn.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/join/FullJoinFn.java @@ -1,8 +1,8 @@ -package cz.seznam.euphoria.beam.join; +package org.apache.beam.sdk.extensions.euphoria.beam.join; -import cz.seznam.euphoria.beam.SingleValueCollector; -import cz.seznam.euphoria.core.client.functional.BinaryFunctor; -import cz.seznam.euphoria.core.client.util.Pair; +import org.apache.beam.sdk.extensions.euphoria.beam.SingleValueCollector; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.BinaryFunctor; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; import org.apache.beam.sdk.transforms.join.CoGbkResult; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/InnerJoinFn.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/join/InnerJoinFn.java similarity index 80% rename from sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/InnerJoinFn.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/join/InnerJoinFn.java index 9aebb8b01d0b9..d73b821686bc9 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/InnerJoinFn.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/join/InnerJoinFn.java @@ -1,8 +1,8 @@ -package cz.seznam.euphoria.beam.join; +package org.apache.beam.sdk.extensions.euphoria.beam.join; -import cz.seznam.euphoria.beam.SingleValueCollector; -import cz.seznam.euphoria.core.client.functional.BinaryFunctor; -import cz.seznam.euphoria.core.client.util.Pair; +import org.apache.beam.sdk.extensions.euphoria.beam.SingleValueCollector; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.BinaryFunctor; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; import org.apache.beam.sdk.transforms.join.CoGbkResult; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/JoinFn.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/join/JoinFn.java similarity index 82% rename from sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/JoinFn.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/join/JoinFn.java index bee8d74ffa9c1..0d027e622bd4c 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/JoinFn.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/join/JoinFn.java @@ -1,7 +1,7 @@ -package cz.seznam.euphoria.beam.join; +package org.apache.beam.sdk.extensions.euphoria.beam.join; -import cz.seznam.euphoria.core.client.functional.BinaryFunctor; -import cz.seznam.euphoria.core.client.util.Pair; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.BinaryFunctor; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.join.CoGbkResult; import org.apache.beam.sdk.values.KV; diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/LeftOuterJoinFn.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/join/LeftOuterJoinFn.java similarity index 82% rename from sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/LeftOuterJoinFn.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/join/LeftOuterJoinFn.java index e0d413f584c9f..acfa317adb18b 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/LeftOuterJoinFn.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/join/LeftOuterJoinFn.java @@ -1,8 +1,8 @@ -package cz.seznam.euphoria.beam.join; +package org.apache.beam.sdk.extensions.euphoria.beam.join; -import cz.seznam.euphoria.beam.SingleValueCollector; -import cz.seznam.euphoria.core.client.functional.BinaryFunctor; -import cz.seznam.euphoria.core.client.util.Pair; +import org.apache.beam.sdk.extensions.euphoria.beam.SingleValueCollector; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.BinaryFunctor; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; import org.apache.beam.sdk.transforms.join.CoGbkResult; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/RightOuterJoinFn.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/join/RightOuterJoinFn.java similarity index 82% rename from sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/RightOuterJoinFn.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/join/RightOuterJoinFn.java index 354cf7b7fa7ab..d1c23bfc598ed 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/RightOuterJoinFn.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/join/RightOuterJoinFn.java @@ -1,8 +1,8 @@ -package cz.seznam.euphoria.beam.join; +package org.apache.beam.sdk.extensions.euphoria.beam.join; -import cz.seznam.euphoria.beam.SingleValueCollector; -import cz.seznam.euphoria.core.client.functional.BinaryFunctor; -import cz.seznam.euphoria.core.client.util.Pair; +import org.apache.beam.sdk.extensions.euphoria.beam.SingleValueCollector; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.BinaryFunctor; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; import org.apache.beam.sdk.transforms.join.CoGbkResult; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/package-info.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/join/package-info.java similarity index 93% rename from sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/package-info.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/join/package-info.java index e47d9dadca8b4..6aa8cffddae3b 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/join/package-info.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/join/package-info.java @@ -19,4 +19,4 @@ /** * {@link cz.seznam.euphoria.core.client.operator.Join} translation centered classes. */ -package cz.seznam.euphoria.beam.join; +package org.apache.beam.sdk.extensions.euphoria.beam.join; diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/BeamWindowing.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/window/BeamWindowing.java similarity index 97% rename from sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/BeamWindowing.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/window/BeamWindowing.java index ca181e7a8a2e9..4aceb4f64f525 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/BeamWindowing.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/window/BeamWindowing.java @@ -13,7 +13,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.beam.window; +package org.apache.beam.sdk.extensions.euphoria.beam.window; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.WindowedElement; diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/UnsupportedWindow.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/window/UnsupportedWindow.java similarity index 96% rename from sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/UnsupportedWindow.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/window/UnsupportedWindow.java index b4594947cdbdf..ad267a53e1e86 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/UnsupportedWindow.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/window/UnsupportedWindow.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package cz.seznam.euphoria.beam.window; +package org.apache.beam.sdk.extensions.euphoria.beam.window; import java.util.Objects; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/WindowingUtils.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/window/WindowingUtils.java similarity index 85% rename from sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/WindowingUtils.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/window/WindowingUtils.java index d546deb5f28d2..60101c48fe27c 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/cz/seznam/euphoria/beam/window/WindowingUtils.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/window/WindowingUtils.java @@ -1,8 +1,8 @@ -package cz.seznam.euphoria.beam.window; +package org.apache.beam.sdk.extensions.euphoria.beam.window; -import cz.seznam.euphoria.core.client.dataset.windowing.Window; -import cz.seznam.euphoria.core.client.dataset.windowing.Windowing; -import cz.seznam.euphoria.core.client.operator.WindowWiseOperator; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; +import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Windowing; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.WindowWiseOperator; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.values.PCollection; import org.joda.time.Duration; diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/JoinTest.java b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/JoinTest.java similarity index 88% rename from sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/JoinTest.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/JoinTest.java index 7164f0d352255..b7b1e8f1b15f1 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/JoinTest.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/JoinTest.java @@ -1,18 +1,18 @@ -package cz.seznam.euphoria.beam; +package org.apache.beam.sdk.extensions.euphoria.beam; import static java.util.Arrays.asList; -import cz.seznam.euphoria.core.client.flow.Flow; -import cz.seznam.euphoria.core.client.functional.BinaryFunctor; -import cz.seznam.euphoria.core.client.io.ListDataSink; -import cz.seznam.euphoria.core.client.io.ListDataSource; -import cz.seznam.euphoria.core.client.operator.FullJoin; -import cz.seznam.euphoria.core.client.operator.Join; -import cz.seznam.euphoria.core.client.operator.LeftJoin; -import cz.seznam.euphoria.core.client.operator.RightJoin; -import cz.seznam.euphoria.core.client.util.Pair; -import cz.seznam.euphoria.testing.DatasetAssert; import java.util.Optional; +import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; +import org.apache.beam.sdk.extensions.euphoria.core.client.functional.BinaryFunctor; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.ListDataSink; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.ListDataSource; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.FullJoin; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.Join; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.LeftJoin; +import org.apache.beam.sdk.extensions.euphoria.core.client.operator.RightJoin; +import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; +import org.apache.beam.sdk.extensions.euphoria.testing.DatasetAssert; import org.junit.Test; /** diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTest.java b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTest.java index 1ff11adc76c15..aec0e8613ca1f 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTest.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTest.java @@ -19,9 +19,7 @@ import static org.junit.Assert.assertTrue; -import cz.seznam.euphoria.beam.TestUtils; -import cz.seznam.euphoria.beam.window.BeamWindowing; -import java.time.Duration; +import org.apache.beam.sdk.extensions.euphoria.beam.window.BeamWindowing; import java.util.Arrays; import java.util.Collections; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; @@ -49,8 +47,6 @@ import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; import org.apache.beam.sdk.extensions.euphoria.core.client.util.Sums; import org.apache.beam.sdk.extensions.euphoria.testing.DatasetAssert; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.windowing.AfterWatermark; import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.values.WindowingStrategy.AccumulationMode; diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/TestUtils.java b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/TestUtils.java similarity index 90% rename from sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/TestUtils.java rename to sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/TestUtils.java index 1cc188ca0e13a..90b63f36e97a5 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/cz/seznam/euphoria/beam/TestUtils.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/TestUtils.java @@ -1,4 +1,4 @@ -package cz.seznam.euphoria.beam; +package org.apache.beam.sdk.extensions.euphoria.beam; import java.time.Duration; import org.apache.beam.sdk.options.PipelineOptions; diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/JoinTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/JoinTest.java index 033754bbc410d..3192b666c62e9 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/JoinTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/JoinTest.java @@ -19,8 +19,8 @@ import static org.junit.Assert.assertEquals; -import java.time.Duration; -import cz.seznam.euphoria.beam.window.BeamWindowing; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import org.apache.beam.sdk.extensions.euphoria.beam.window.BeamWindowing; import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -29,11 +29,7 @@ import java.util.Optional; import org.apache.beam.sdk.extensions.euphoria.beam.io.KryoCoder; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; -import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Session; -import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.TimeInterval; -import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.WindowedElement; -import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Windowing; import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; import org.apache.beam.sdk.extensions.euphoria.core.client.io.ListDataSource; @@ -43,9 +39,6 @@ import org.apache.beam.sdk.extensions.euphoria.core.client.operator.LeftJoin; import org.apache.beam.sdk.extensions.euphoria.core.client.operator.MapElements; import org.apache.beam.sdk.extensions.euphoria.core.client.operator.RightJoin; -import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.NoopTrigger; -import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.Trigger; -import org.apache.beam.sdk.extensions.euphoria.core.client.util.Either; import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; import org.apache.beam.sdk.extensions.euphoria.core.client.util.Triple; import org.apache.beam.sdk.extensions.euphoria.operator.test.accumulators.SnapshotProvider; diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/ReduceByKeyTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/ReduceByKeyTest.java index 95040a12e4bae..87ea133523eb1 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/ReduceByKeyTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/ReduceByKeyTest.java @@ -23,8 +23,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Sets; -import java.time.Duration; -import cz.seznam.euphoria.beam.window.BeamWindowing; +import org.apache.beam.sdk.extensions.euphoria.beam.window.BeamWindowing; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -40,10 +39,7 @@ import org.apache.beam.sdk.extensions.euphoria.beam.io.KryoCoder; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Count; -import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.GlobalWindowing; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.MergingWindowing; -import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Session; -import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Time; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.TimeInterval; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.WindowedElement; @@ -59,7 +55,6 @@ import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.StateContext; import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ValueStorage; import org.apache.beam.sdk.extensions.euphoria.core.client.operator.state.ValueStorageDescriptor; -import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.CountTrigger; import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.NoopTrigger; import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.Trigger; import org.apache.beam.sdk.extensions.euphoria.core.client.triggers.TriggerContext; From 22810d117cb8c7741f583c799af3122cccfb73f9 Mon Sep 17 00:00:00 2001 From: Vaclav Plajt Date: Mon, 21 May 2018 09:19:07 +0200 Subject: [PATCH 11/13] [BEAM-4294] [BEAM-4360] Checkstyle warning fixes. --- .../beam/sdk/extensions/euphoria/beam/JoinTranslator.java | 8 ++++---- .../extensions/euphoria/beam/ReduceByKeyTranslator.java | 2 +- .../extensions/euphoria/beam/window/BeamWindowing.java | 3 ++- .../sdk/extensions/euphoria/beam/ReduceByKeyTest.java | 2 +- .../euphoria/operator/test/ReduceByKeyTest.java | 6 +++--- 5 files changed, 11 insertions(+), 10 deletions(-) diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/JoinTranslator.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/JoinTranslator.java index 888aaf0996c77..919cba661bd47 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/JoinTranslator.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/JoinTranslator.java @@ -1,16 +1,16 @@ package org.apache.beam.sdk.extensions.euphoria.beam; +import java.util.List; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.extensions.euphoria.beam.common.InputToKvDoFn; +import org.apache.beam.sdk.extensions.euphoria.beam.io.KryoCoder; import org.apache.beam.sdk.extensions.euphoria.beam.join.FullJoinFn; import org.apache.beam.sdk.extensions.euphoria.beam.join.InnerJoinFn; import org.apache.beam.sdk.extensions.euphoria.beam.join.JoinFn; import org.apache.beam.sdk.extensions.euphoria.beam.join.LeftOuterJoinFn; import org.apache.beam.sdk.extensions.euphoria.beam.join.RightOuterJoinFn; import org.apache.beam.sdk.extensions.euphoria.beam.window.WindowingUtils; -import java.util.List; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.KvCoder; -import org.apache.beam.sdk.extensions.euphoria.beam.io.KryoCoder; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; import org.apache.beam.sdk.extensions.euphoria.core.client.functional.BinaryFunctor; import org.apache.beam.sdk.extensions.euphoria.core.client.functional.UnaryFunction; diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTranslator.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTranslator.java index 5e74ebff67d97..0e0ddef7de066 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTranslator.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTranslator.java @@ -18,11 +18,11 @@ package org.apache.beam.sdk.extensions.euphoria.beam; -import org.apache.beam.sdk.extensions.euphoria.beam.window.WindowingUtils; import java.util.stream.StreamSupport; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.extensions.euphoria.beam.window.WindowingUtils; import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.AccumulatorProvider; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; import org.apache.beam.sdk.extensions.euphoria.core.client.functional.ReduceFunctor; diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/window/BeamWindowing.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/window/BeamWindowing.java index 4aceb4f64f525..c441a3fb486c2 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/window/BeamWindowing.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/window/BeamWindowing.java @@ -57,7 +57,8 @@ public Iterable assignWindowsToElement(WindowedElement } @Override - public org.apache.beam.sdk.extensions.euphoria.core.client.triggers.Trigger getTrigger() { + public org.apache.beam.sdk.extensions.euphoria.core.client.triggers.Trigger + getTrigger() { throw new UnsupportedOperationException( "Beam window serves as envelope, it do not contains Euphoria trigger."); } diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTest.java b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTest.java index aec0e8613ca1f..519e94fe976a2 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTest.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTest.java @@ -19,9 +19,9 @@ import static org.junit.Assert.assertTrue; -import org.apache.beam.sdk.extensions.euphoria.beam.window.BeamWindowing; import java.util.Arrays; import java.util.Collections; +import org.apache.beam.sdk.extensions.euphoria.beam.window.BeamWindowing; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.TimeInterval; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Window; diff --git a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/ReduceByKeyTest.java b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/ReduceByKeyTest.java index 87ea133523eb1..fc6f1e248e250 100644 --- a/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/ReduceByKeyTest.java +++ b/sdks/java/extensions/euphoria/euphoria-operator-testkit/src/main/java/org/apache/beam/sdk/extensions/euphoria/operator/test/ReduceByKeyTest.java @@ -23,7 +23,6 @@ import com.google.common.collect.Lists; import com.google.common.collect.Sets; -import org.apache.beam.sdk.extensions.euphoria.beam.window.BeamWindowing; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -36,7 +35,10 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import java.util.stream.Stream; +import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.extensions.euphoria.beam.io.KryoCoder; +import org.apache.beam.sdk.extensions.euphoria.beam.window.BeamWindowing; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.Dataset; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.Count; import org.apache.beam.sdk.extensions.euphoria.core.client.dataset.windowing.MergingWindowing; @@ -66,8 +68,6 @@ import org.apache.beam.sdk.extensions.euphoria.operator.test.accumulators.SnapshotProvider; import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.AbstractOperatorTest; import org.apache.beam.sdk.extensions.euphoria.operator.test.junit.Processing; -import javax.annotation.Nullable; -import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.windowing.AfterWatermark; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.FixedWindows; From 013b0783308fcd03d269b9fa1bf6c26e251d6169 Mon Sep 17 00:00:00 2001 From: Vaclav Plajt Date: Mon, 21 May 2018 12:27:19 +0200 Subject: [PATCH 12/13] [BEAM-4394] Pull request review fixes. --- .../euphoria/beam/join/FullJoinFn.java | 26 +++++++------------ .../euphoria/beam/join/InnerJoinFn.java | 13 +++------- .../extensions/euphoria/beam/join/JoinFn.java | 17 +++++++++++- .../euphoria/beam/join/LeftOuterJoinFn.java | 23 ++++++---------- .../euphoria/beam/join/RightOuterJoinFn.java | 13 +++------- .../euphoria/beam/join/package-info.java | 3 ++- 6 files changed, 43 insertions(+), 52 deletions(-) diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/join/FullJoinFn.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/join/FullJoinFn.java index 8333c41d18aef..ec694c70766c2 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/join/FullJoinFn.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/join/FullJoinFn.java @@ -4,7 +4,6 @@ import org.apache.beam.sdk.extensions.euphoria.core.client.functional.BinaryFunctor; import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; import org.apache.beam.sdk.transforms.join.CoGbkResult; -import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; /** @@ -18,39 +17,34 @@ public FullJoinFn(BinaryFunctor joiner, TupleTag } @Override - public void processElement(ProcessContext c) { + protected void doJoin( + ProcessContext c, K key, CoGbkResult value, + Iterable leftSideIter, + Iterable rightSideIter) { - KV element = c.element(); - CoGbkResult value = element.getValue(); - K key = element.getKey(); - - Iterable leftSideIter = value.getAll(leftTag); - Iterable rightSIdeIter = value.getAll(rightTag); + boolean leftHasValues = leftSideIter.iterator().hasNext(); + boolean rightHasValues = rightSideIter.iterator().hasNext(); SingleValueCollector outCollector = new SingleValueCollector<>(); - boolean leftHasValues = leftSideIter.iterator().hasNext(); - boolean rightHasValues = rightSIdeIter.iterator().hasNext(); - if (leftHasValues && rightHasValues) { - for (RightT rightValue : rightSIdeIter) { + for (RightT rightValue : rightSideIter) { for (LeftT leftValue : leftSideIter) { joiner.apply(leftValue, rightValue, outCollector); c.output(Pair.of(key, outCollector.get())); } } - } else if (leftHasValues && !rightHasValues) { + } else if (leftHasValues) { for (LeftT leftValue : leftSideIter) { joiner.apply(leftValue, null, outCollector); c.output(Pair.of(key, outCollector.get())); } - } else if (!leftHasValues && rightHasValues) { - for (RightT rightValue : rightSIdeIter) { + } else if (rightHasValues) { + for (RightT rightValue : rightSideIter) { joiner.apply(null, rightValue, outCollector); c.output(Pair.of(key, outCollector.get())); } } - } @Override diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/join/InnerJoinFn.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/join/InnerJoinFn.java index d73b821686bc9..b5a58a0b7f037 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/join/InnerJoinFn.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/join/InnerJoinFn.java @@ -4,7 +4,6 @@ import org.apache.beam.sdk.extensions.euphoria.core.client.functional.BinaryFunctor; import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; import org.apache.beam.sdk.transforms.join.CoGbkResult; -import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; /** @@ -20,14 +19,10 @@ public InnerJoinFn( } @Override - public void processElement(ProcessContext c) { - - KV element = c.element(); - CoGbkResult value = element.getValue(); - K key = element.getKey(); - - Iterable leftSideIter = value.getAll(leftTag); - Iterable rightSideIter = value.getAll(rightTag); + protected void doJoin( + ProcessContext c, K key, CoGbkResult value, + Iterable leftSideIter, + Iterable rightSideIter) { SingleValueCollector outCollector = new SingleValueCollector<>(); diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/join/JoinFn.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/join/JoinFn.java index 0d027e622bd4c..ee9a6116808db 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/join/JoinFn.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/join/JoinFn.java @@ -31,7 +31,22 @@ protected JoinFn( } @ProcessElement - public abstract void processElement(ProcessContext c); + public final void processElement(ProcessContext c) { + + KV element = c.element(); + CoGbkResult value = element.getValue(); + K key = element.getKey(); + + Iterable leftSideIter = value.getAll(leftTag); + Iterable rightSideIter = value.getAll(rightTag); + + doJoin(c, key, value, leftSideIter, rightSideIter); + } + + protected abstract void doJoin( + ProcessContext c, K key, CoGbkResult value, + Iterable leftSideIter, + Iterable rightSideIter); public abstract String getFnName(); } diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/join/LeftOuterJoinFn.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/join/LeftOuterJoinFn.java index acfa317adb18b..102666a428397 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/join/LeftOuterJoinFn.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/join/LeftOuterJoinFn.java @@ -4,7 +4,6 @@ import org.apache.beam.sdk.extensions.euphoria.core.client.functional.BinaryFunctor; import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; import org.apache.beam.sdk.transforms.join.CoGbkResult; -import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; @@ -21,30 +20,24 @@ public LeftOuterJoinFn( } @Override - public void processElement(ProcessContext c) { - - KV element = c.element(); - CoGbkResult value = element.getValue(); - K key = element.getKey(); - - Iterable leftSideIter = value.getAll(leftTag); - Iterable rightSIdeIter = value.getAll(rightTag); + protected void doJoin( + ProcessContext c, K key, CoGbkResult value, + Iterable leftSideIter, + Iterable rightSideIter) { SingleValueCollector outCollector = new SingleValueCollector<>(); for (LeftT leftValue : leftSideIter) { - if (rightSIdeIter.iterator().hasNext()) { - for (RightT rightValue : rightSIdeIter) { + if (rightSideIter.iterator().hasNext()) { + for (RightT rightValue : rightSideIter) { joiner.apply(leftValue, rightValue, outCollector); c.output(Pair.of(key, outCollector.get())); } } else { - joiner.apply(leftValue, null, outCollector); - c.output(Pair.of(key, outCollector.get())); + joiner.apply(leftValue, null, outCollector); + c.output(Pair.of(key, outCollector.get())); } } - - } @Override diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/join/RightOuterJoinFn.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/join/RightOuterJoinFn.java index d1c23bfc598ed..9f0e8ce5ccdf2 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/join/RightOuterJoinFn.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/join/RightOuterJoinFn.java @@ -4,7 +4,6 @@ import org.apache.beam.sdk.extensions.euphoria.core.client.functional.BinaryFunctor; import org.apache.beam.sdk.extensions.euphoria.core.client.util.Pair; import org.apache.beam.sdk.transforms.join.CoGbkResult; -import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; /** @@ -20,18 +19,12 @@ public RightOuterJoinFn( } @Override - public void processElement(ProcessContext c) { - - KV element = c.element(); - CoGbkResult value = element.getValue(); - K key = element.getKey(); - - Iterable leftSideIter = value.getAll(leftTag); - Iterable rightSIdeIter = value.getAll(rightTag); + protected void doJoin(ProcessContext c, K key, CoGbkResult value, Iterable leftSideIter, + Iterable rightSideIter) { SingleValueCollector outCollector = new SingleValueCollector<>(); - for (RightT rightValue : rightSIdeIter) { + for (RightT rightValue : rightSideIter) { if (leftSideIter.iterator().hasNext()) { for (LeftT leftValue : leftSideIter) { joiner.apply(leftValue, rightValue, outCollector); diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/join/package-info.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/join/package-info.java index 6aa8cffddae3b..8548a4a126553 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/join/package-info.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/join/package-info.java @@ -17,6 +17,7 @@ */ /** - * {@link cz.seznam.euphoria.core.client.operator.Join} translation centered classes. + * {@link org.apache.beam.sdk.extensions.euphoria.core.client.operator.Join} translation centered + * classes. */ package org.apache.beam.sdk.extensions.euphoria.beam.join; From 9600446bfc7e092b3e5156f171fa4147358c1e4b Mon Sep 17 00:00:00 2001 From: Vaclav Plajt Date: Mon, 21 May 2018 14:49:29 +0200 Subject: [PATCH 13/13] [BEAM-4294] [BEAM-4360] Pull request review fixes. OperatorTranslator extended to allow more than one translator per operator. --- .../euphoria/beam/FlowTranslator.java | 49 ++++++++++++++++--- .../euphoria/beam/JoinTranslator.java | 6 +-- .../euphoria/beam/OperatorTranslator.java | 12 ++++- .../euphoria/beam/ReduceByKeyTranslator.java | 13 +++-- .../euphoria/beam/common/InputToKvDoFn.java | 4 +- .../extensions/euphoria/beam/JoinTest.java | 35 ++++++------- 6 files changed, 83 insertions(+), 36 deletions(-) diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/FlowTranslator.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/FlowTranslator.java index 25891bb3afb6b..581abec9f46d8 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/FlowTranslator.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/FlowTranslator.java @@ -17,8 +17,10 @@ */ package org.apache.beam.sdk.extensions.euphoria.beam; -import java.util.IdentityHashMap; -import java.util.Map; +import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.Multimap; +import java.util.Collection; +import javax.annotation.Nullable; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.extensions.euphoria.beam.io.BeamWriteSink; import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.AccumulatorProvider; @@ -44,8 +46,10 @@ */ class FlowTranslator { - private static final Map translators = new IdentityHashMap<>(); + private static final Multimap translators = ArrayListMultimap.create(); + //Note that when there are more than one translator ordering defines priority. + //First added to `translators` is first asked whenever it can translate the operator. static { translators.put(FlowUnfolder.InputOperator.class, new InputTranslator()); translators.put(FlatMap.class, new FlatMapTranslator()); @@ -58,6 +62,39 @@ class FlowTranslator { translators.put(Join.class, new JoinTranslator()); } + @SuppressWarnings("unchecked") + private static boolean isOperatorDirectlyTranslatable(Operator operator){ + Collection availableTranslators = translators.get(operator.getClass()); + if (availableTranslators.isEmpty()){ + return false; + } + + for (OperatorTranslator translator : availableTranslators){ + if (translator.canTranslate(operator)){ + return true; + } + } + + return false; + } + + @Nullable + @SuppressWarnings("unchecked") + private static OperatorTranslator getTranslatorIfAvailable(Operator operator){ + Collection availableTranslators = translators.get(operator.getClass()); + if (availableTranslators.isEmpty()){ + return null; + } + + for (OperatorTranslator translator : availableTranslators){ + if (translator.canTranslate(operator)){ + return translator; + } + } + + return null; + } + static Pipeline toPipeline( Flow flow, AccumulatorProvider.Factory accumulatorFactory, @@ -77,12 +114,12 @@ static Pipeline toPipeline( static DAG> toDAG(Flow flow) { final DAG> dag = - FlowUnfolder.unfold(flow, operator -> translators.containsKey(operator.getClass())); + FlowUnfolder.unfold(flow, FlowTranslator::isOperatorDirectlyTranslatable); return dag; } static DAG> unfold(DAG> dag) { - return FlowUnfolder.translate(dag, operator -> translators.containsKey(operator.getClass())); + return FlowUnfolder.translate(dag, FlowTranslator::isOperatorDirectlyTranslatable); } @SuppressWarnings("unchecked") @@ -93,7 +130,7 @@ static void updateContextBy(DAG> dag, BeamExecutorContext context .map(Node::get) .forEach( op -> { - final OperatorTranslator translator = translators.get(op.getClass()); + final OperatorTranslator translator = getTranslatorIfAvailable(op); if (translator == null) { throw new UnsupportedOperationException( "Operator " + op.getClass().getSimpleName() + " not supported"); diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/JoinTranslator.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/JoinTranslator.java index 919cba661bd47..064b8ae1086c4 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/JoinTranslator.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/JoinTranslator.java @@ -85,11 +85,11 @@ private PCollection> getKVInputCollection( PCollection typedInput = (PCollection) inputPCollection; typedInput.setCoder(valueCoder); - PCollection> leftKvInput = + PCollection> kvInput = typedInput.apply(transformName, ParDo.of(new InputToKvDoFn<>(keyExtractor))); - leftKvInput.setCoder(KvCoder.of(keyCoder, valueCoder)); + kvInput.setCoder(KvCoder.of(keyCoder, valueCoder)); - return leftKvInput; + return kvInput; } private > JoinFn diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/OperatorTranslator.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/OperatorTranslator.java index a184fc973b677..f24f3790a5582 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/OperatorTranslator.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/OperatorTranslator.java @@ -25,7 +25,6 @@ * * @param the type of the user defined euphoria operator definition */ -@FunctionalInterface interface OperatorTranslator { /** @@ -36,4 +35,15 @@ interface OperatorTranslator { * @return a beam transformation */ PCollection translate(OperatorT operator, BeamExecutorContext context); + + /** + * Returns true when implementing {@link OperatorTranslator} is able to translate given instance + * of an operator, false otherwise. + * + *

This method allow us to have more {@link OperatorTranslator} + * implementations for one {@link Operator} in case when some specialized translators are needed. + */ + default boolean canTranslate(OperatorT operator) { + return true; + } } diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTranslator.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTranslator.java index 0e0ddef7de066..382f2189ee552 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTranslator.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/ReduceByKeyTranslator.java @@ -18,6 +18,8 @@ package org.apache.beam.sdk.extensions.euphoria.beam; +import static com.google.common.base.Preconditions.checkState; + import java.util.stream.StreamSupport; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.IterableCoder; @@ -48,9 +50,8 @@ class ReduceByKeyTranslator implements OperatorTranslator { private static > PCollection> doTranslate(ReduceByKey operator, BeamExecutorContext context) { - if (operator.getValueComparator() != null) { //TODO Could we even do values sorting ? - throw new UnsupportedOperationException("Values sorting is not supported."); - } + //TODO Could we even do values sorting ? + checkState(operator.getValueComparator() == null, "Values sorting is not supported."); final UnaryFunction keyExtractor = operator.getKeyExtractor(); final UnaryFunction valueExtractor = operator.getValueExtractor(); @@ -107,6 +108,12 @@ public Pair apply(KV in) { } } + @Override + public boolean canTranslate(ReduceByKey operator) { + // translation of sorted values is not supported yet + return operator.getValueComparator() == null; + } + private static SerializableFunction, InputT> asCombiner( ReduceFunctor reducer) { diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/common/InputToKvDoFn.java b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/common/InputToKvDoFn.java index 83670856352f9..49420cdd9f3b8 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/common/InputToKvDoFn.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/main/java/org/apache/beam/sdk/extensions/euphoria/beam/common/InputToKvDoFn.java @@ -12,8 +12,8 @@ public class InputToKvDoFn extends DoFn> { private final UnaryFunction keyExtractor; - public InputToKvDoFn(UnaryFunction leftKeyExtractor) { - this.keyExtractor = leftKeyExtractor; + public InputToKvDoFn(UnaryFunction keyExtractor) { + this.keyExtractor = keyExtractor; } @ProcessElement diff --git a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/JoinTest.java b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/JoinTest.java index b7b1e8f1b15f1..278fac1bdb315 100644 --- a/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/JoinTest.java +++ b/sdks/java/extensions/euphoria/euphoria-beam/src/test/java/org/apache/beam/sdk/extensions/euphoria/beam/JoinTest.java @@ -4,7 +4,7 @@ import java.util.Optional; import org.apache.beam.sdk.extensions.euphoria.core.client.flow.Flow; -import org.apache.beam.sdk.extensions.euphoria.core.client.functional.BinaryFunctor; +import org.apache.beam.sdk.extensions.euphoria.core.client.io.Collector; import org.apache.beam.sdk.extensions.euphoria.core.client.io.ListDataSink; import org.apache.beam.sdk.extensions.euphoria.core.client.io.ListDataSource; import org.apache.beam.sdk.extensions.euphoria.core.client.operator.FullJoin; @@ -42,12 +42,11 @@ public void simpleInnerJoinTest() { ListDataSink>> output = ListDataSink.get(); - BinaryFunctor, Pair, Pair> joiner = - (l, r, c) -> c.collect(Pair.of(l.getSecond(), r.getSecond())); - Join.of(flow.createInput(left), flow.createInput(right)) .by(Pair::getFirst, Pair::getFirst) - .using(joiner) + .using( + (Pair l, Pair r, Collector> c) + -> c.collect(Pair.of(l.getSecond(), r.getSecond()))) .output() .persist(output); @@ -85,13 +84,11 @@ public void simpleLeftJoinTest() { ListDataSink>> output = ListDataSink.get(); - BinaryFunctor, Optional>, Pair> - joiner = (l, r, c) -> - c.collect(Pair.of(l.getSecond(), r.orElse(Pair.of(null, null)).getSecond())); - LeftJoin.of(flow.createInput(left), flow.createInput(right)) .by(Pair::getFirst, Pair::getFirst) - .using(joiner) + .using((Pair l, Optional> r, + Collector> c) -> + c.collect(Pair.of(l.getSecond(), r.orElse(Pair.of(null, null)).getSecond()))) .output() .persist(output); @@ -131,13 +128,12 @@ public void simpleRightJoinTest() { ListDataSink>> output = ListDataSink.get(); - BinaryFunctor>, Pair, Pair> - joiner = (l, r, c) -> - c.collect(Pair.of(l.orElse(Pair.of(null, null)).getSecond(), r.getSecond())); - RightJoin.of(flow.createInput(left), flow.createInput(right)) .by(Pair::getFirst, Pair::getFirst) - .using(joiner) + .using( + (Optional> l, Pair r, + Collector> c) -> + c.collect(Pair.of(l.orElse(Pair.of(null, null)).getSecond(), r.getSecond()))) .output() .persist(output); @@ -177,14 +173,11 @@ public void simpleFullJoinTest() { ListDataSink>> output = ListDataSink.get(); - BinaryFunctor< - Optional>, Optional>, Pair> - joiner = (l, r, c) -> c.collect(Pair.of( - l.orElse(Pair.of(null, null)).getSecond(), r.orElse(Pair.of(null, null)).getSecond())); - FullJoin.of(flow.createInput(left), flow.createInput(right)) .by(Pair::getFirst, Pair::getFirst) - .using(joiner) + .using((Optional> l, Optional> r, + Collector> c) -> c.collect(Pair.of( + l.orElse(Pair.of(null, null)).getSecond(), r.orElse(Pair.of(null, null)).getSecond()))) .output() .persist(output);