-
Notifications
You must be signed in to change notification settings - Fork 0
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[BEAM-4294] Join operator translator #2
Changes from all commits
dc00d9a
11920f0
c10d846
a5bfb38
06ca634
f353852
0f8f364
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,125 @@ | ||
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; | ||
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; | ||
|
||
|
||
/** | ||
* {@link OperatorTranslator Translator } for Euphoria {@link Join} operator. | ||
*/ | ||
public class JoinTranslator implements OperatorTranslator<Join> { | ||
|
||
@Override | ||
@SuppressWarnings("unchecked") | ||
public PCollection<?> translate(Join operator, BeamExecutorContext context) { | ||
return doTranslate(operator, context); | ||
} | ||
|
||
|
||
public <K, LeftT, RightT, OutputT, W extends Window<W>> PCollection<Pair<K, OutputT>> | ||
doTranslate(Join<LeftT, RightT, K, OutputT, W> operator, BeamExecutorContext context) { | ||
|
||
Coder<K> keyCoder = context.getCoder(operator.getLeftKeyExtractor()); | ||
|
||
// get input data-sets transformed to Pcollections<KV<K,LeftT/RightT>> | ||
List<PCollection<Object>> inputs = context.getInputs(operator); | ||
|
||
PCollection<KV<K, LeftT>> leftKvInput = getKVInputCollection(inputs.get(0), | ||
operator.getLeftKeyExtractor(), | ||
keyCoder, new KryoCoder<>(), "::extract-keys-left"); | ||
|
||
PCollection<KV<K, RightT>> rightKvInput = getKVInputCollection(inputs.get(1), | ||
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<LeftT> leftTag = new TupleTag<>(); | ||
TupleTag<RightT> rightTag = new TupleTag<>(); | ||
|
||
PCollection<KV<K, CoGbkResult>> coGrouped = KeyedPCollectionTuple | ||
.of(leftTag, leftKvInput) | ||
.and(rightTag, rightKvInput) | ||
.apply("::co-group-by-key", CoGroupByKey.create()); | ||
|
||
// Join | ||
JoinFn<LeftT, RightT, K, OutputT> joinFn = chooseJoinFn(operator, leftTag, rightTag); | ||
|
||
return coGrouped.apply(joinFn.getFnName(), ParDo.of(joinFn)); | ||
} | ||
|
||
private <K, ValueT> PCollection<KV<K, ValueT>> getKVInputCollection( | ||
PCollection<Object> inputPCollection, | ||
UnaryFunction<ValueT, K> keyExtractor, | ||
Coder<K> keyCoder, Coder<ValueT> valueCoder, String transformName) { | ||
|
||
@SuppressWarnings("unchecked") | ||
PCollection<ValueT> typedInput = (PCollection<ValueT>) inputPCollection; | ||
typedInput.setCoder(valueCoder); | ||
|
||
PCollection<KV<K, ValueT>> leftKvInput = | ||
typedInput.apply(transformName, ParDo.of(new InputToKvDoFn<>(keyExtractor))); | ||
leftKvInput.setCoder(KvCoder.of(keyCoder, valueCoder)); | ||
|
||
return leftKvInput; | ||
} | ||
|
||
private <K, LeftT, RightT, OutputT, W extends Window<W>> JoinFn<LeftT, RightT, K, OutputT> | ||
chooseJoinFn( | ||
Join<LeftT, RightT, K, OutputT, W> operator, TupleTag<LeftT> leftTag, | ||
TupleTag<RightT> rightTag) { | ||
|
||
JoinFn<LeftT, RightT, K, OutputT> joinFn; | ||
BinaryFunctor<LeftT, RightT, OutputT> joiner = operator.getJoiner(); | ||
|
||
switch (operator.getType()) { | ||
case INNER: | ||
joinFn = new InnerJoinFn<>(joiner, leftTag, rightTag); | ||
break; | ||
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." | ||
+ " Given join type '%s' is not supported.", | ||
Join.class.getSimpleName(), operator.getType())); | ||
} | ||
return joinFn; | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,26 @@ | ||
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; | ||
|
||
/** | ||
* {@link DoFn} which takes input elements and transforms them to {@link KV} using given key | ||
* extractor. | ||
*/ | ||
public class InputToKvDoFn<InputT, K> extends DoFn<InputT, KV<K, InputT>> { | ||
|
||
private final UnaryFunction<InputT, K> keyExtractor; | ||
|
||
public InputToKvDoFn(UnaryFunction<InputT, K> 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)); | ||
} | ||
|
||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -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; |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -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<LeftT, RightT, K, OutputT> extends JoinFn<LeftT, RightT, K, OutputT> { | ||
|
||
public FullJoinFn(BinaryFunctor<LeftT, RightT, OutputT> joiner, TupleTag<LeftT> leftTag, | ||
TupleTag<RightT> rightTag) { | ||
super(joiner, leftTag, rightTag); | ||
} | ||
|
||
@Override | ||
public void processElement(ProcessContext c) { | ||
|
||
KV<K, CoGbkResult> element = c.element(); | ||
CoGbkResult value = element.getValue(); | ||
K key = element.getKey(); | ||
|
||
Iterable<LeftT> leftSideIter = value.getAll(leftTag); | ||
Iterable<RightT> rightSIdeIter = value.getAll(rightTag); | ||
|
||
SingleValueCollector<OutputT> 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) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Fixed in seznam@013b078 This pull request will be closed. |
||
for (LeftT leftValue : leftSideIter) { | ||
joiner.apply(leftValue, null, outCollector); | ||
c.output(Pair.of(key, outCollector.get())); | ||
} | ||
} else if (!leftHasValues && rightHasValues) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. !leftHasValues is always true, you can delete it There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Fixed in seznam@013b078 This pull request will be closed. |
||
for (RightT rightValue : rightSIdeIter) { | ||
joiner.apply(null, rightValue, outCollector); | ||
c.output(Pair.of(key, outCollector.get())); | ||
} | ||
} | ||
|
||
} | ||
|
||
@Override | ||
public String getFnName() { | ||
return "::full-join"; | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,47 @@ | ||
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; | ||
|
||
/** | ||
* Inner join implementation of {@link JoinFn}. | ||
*/ | ||
public class InnerJoinFn<LeftT, RightT, K, OutputT> extends JoinFn<LeftT, RightT, K, OutputT> { | ||
|
||
public InnerJoinFn( | ||
BinaryFunctor<LeftT, RightT, OutputT> functor, | ||
TupleTag<LeftT> leftTag, | ||
TupleTag<RightT> rightTag) { | ||
super(functor, leftTag, rightTag); | ||
} | ||
|
||
@Override | ||
public void processElement(ProcessContext c) { | ||
|
||
KV<K, CoGbkResult> element = c.element(); | ||
CoGbkResult value = element.getValue(); | ||
K key = element.getKey(); | ||
|
||
Iterable<LeftT> leftSideIter = value.getAll(leftTag); | ||
Iterable<RightT> rightSideIter = value.getAll(rightTag); | ||
|
||
SingleValueCollector<OutputT> 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"; | ||
} | ||
|
||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
variable typo. lowercase
i
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Fixed in seznam@013b078 This pull request will be closed.