Skip to content
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

Reapply "Add Redistribute translation to Samza runner" #31270

Merged
merged 2 commits into from
May 15, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
3 changes: 2 additions & 1 deletion .github/trigger_files/beam_PostCommit_Java_PVR_Samza.json
Original file line number Diff line number Diff line change
@@ -1,4 +1,5 @@
{
"comment": "Modify this file in a trivial way to cause this test suite to run",
"https://github.com/apache/beam/pull/31156": "noting that PR #31156 should run this test"
"https://github.com/apache/beam/pull/31156": "noting that PR #31156 should run this test",
"https://github.com/apache/beam/pull/31270": "re-adds specialized Samza translation of Redistribute"
}
Original file line number Diff line number Diff line change
@@ -1,4 +1,5 @@
{
"comment": "Modify this file in a trivial way to cause this test suite to run",
"https://github.com/apache/beam/pull/31156": "noting that PR #31156 should run this test"
"https://github.com/apache/beam/pull/31156": "noting that PR #31156 should run this test",
"https://github.com/apache/beam/pull/31270": "re-add specialized Samza translation of Redistribute"
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,64 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.beam.runners.samza.translation;

import com.google.auto.service.AutoService;
import org.apache.beam.model.pipeline.v1.RunnerApi;
import org.apache.beam.sdk.runners.TransformHierarchy;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.util.construction.NativeTransforms;
import org.apache.beam.sdk.util.construction.graph.PipelineNode;
import org.apache.beam.sdk.util.construction.graph.QueryablePipeline;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollection;

/**
* Translates Reshuffle transform into Samza's native partitionBy operator, which will partition
* each incoming message by the key into a Task corresponding to that key.
*/
public class RedistributeByKeyTranslator<K, V>
implements TransformTranslator<PTransform<PCollection<KV<K, V>>, PCollection<KV<K, V>>>> {

private final ReshuffleTranslator<K, V, V> reshuffleTranslator =
new ReshuffleTranslator<>("rdstr-");

@Override
public void translate(
PTransform<PCollection<KV<K, V>>, PCollection<KV<K, V>>> transform,
TransformHierarchy.Node node,
TranslationContext ctx) {
reshuffleTranslator.translate(transform, node, ctx);
}

@Override
public void translatePortable(
PipelineNode.PTransformNode transform,
QueryablePipeline pipeline,
PortableTranslationContext ctx) {
reshuffleTranslator.translatePortable(transform, pipeline, ctx);
}

/** Predicate to determine whether a URN is a Samza native transform. */
@AutoService(NativeTransforms.IsNativeTransform.class)
public static class IsSamzaNativeTransform implements NativeTransforms.IsNativeTransform {
@Override
public boolean test(RunnerApi.PTransform pTransform) {
return false;
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,16 @@
public class ReshuffleTranslator<K, InT, OutT>
implements TransformTranslator<PTransform<PCollection<KV<K, InT>>, PCollection<KV<K, OutT>>>> {

private final String prefix;

ReshuffleTranslator(String prefix) {
this.prefix = prefix;
}

ReshuffleTranslator() {
this("rshfl-");
}

@Override
public void translate(
PTransform<PCollection<KV<K, InT>>, PCollection<KV<K, OutT>>> transform,
Expand All @@ -60,7 +70,7 @@ public void translate(
inputStream,
inputCoder.getKeyCoder(),
elementCoder,
"rshfl-" + ctx.getTransformId(),
prefix + ctx.getTransformId(),
ctx.getPipelineOptions().getMaxSourceParallelism() > 1);

ctx.registerMessageStream(output, outputStream);
Expand All @@ -83,7 +93,7 @@ public void translatePortable(
inputStream,
((KvCoder<K, InT>) windowedInputCoder.getValueCoder()).getKeyCoder(),
windowedInputCoder,
"rshfl-" + ctx.getTransformId(),
prefix + ctx.getTransformId(),
ctx.getPipelineOptions().getMaxSourceParallelism() > 1);

ctx.registerMessageStream(outputId, outputStream);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -179,6 +179,7 @@ public Map<String, TransformTranslator<?>> getTransformTranslators() {
return ImmutableMap.<String, TransformTranslator<?>>builder()
.put(PTransformTranslation.READ_TRANSFORM_URN, new ReadTranslator<>())
.put(PTransformTranslation.RESHUFFLE_URN, new ReshuffleTranslator<>())
.put(PTransformTranslation.REDISTRIBUTE_BY_KEY_URN, new RedistributeByKeyTranslator<>())
.put(PTransformTranslation.PAR_DO_TRANSFORM_URN, new ParDoBoundMultiTranslator<>())
.put(PTransformTranslation.GROUP_BY_KEY_TRANSFORM_URN, new GroupByKeyTranslator<>())
.put(PTransformTranslation.COMBINE_PER_KEY_TRANSFORM_URN, new GroupByKeyTranslator<>())
Expand Down
Loading