-
Notifications
You must be signed in to change notification settings - Fork 4.3k
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
Add various utility meta-transforms to Beam. #32445
Changes from 6 commits
81a50f2
71d97b6
efcf1ba
acd52f9
177c9ab
7c6ee3e
621cdfb
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,91 @@ | ||
/* | ||
* 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.sdk.transforms; | ||
|
||
import java.util.function.Consumer; | ||
import org.apache.beam.sdk.values.PCollection; | ||
import org.apache.beam.sdk.values.PCollectionTuple; | ||
|
||
/** | ||
* A PTransform that returns its input, but also applies its input to an auxiliary PTransform, akin | ||
* to the shell {@code tee} command, which is named after the T-splitter used in plumbing. | ||
* | ||
* <p>This can be useful to write out or otherwise process an intermediate transform without | ||
* breaking the linear flow of a chain of transforms, e.g. | ||
* | ||
* <pre><code> | ||
* {@literal PCollection<T>} input = ... ; | ||
* {@literal PCollection<T>} result = | ||
* {@literal input.apply(...)} | ||
* ... | ||
* {@literal input.apply(Tee.of(someSideTransform)} | ||
* ... | ||
* {@literal input.apply(...)}; | ||
* </code></pre> | ||
* | ||
* @param <T> the element type of the input PCollection | ||
*/ | ||
public class Tee<T> extends PTransform<PCollection<T>, PCollection<T>> { | ||
private final PTransform<PCollection<T>, ?> consumer; | ||
|
||
/** | ||
* Returns a new Tee PTransform that will apply an auxilary transform to the input as well as pass | ||
* it on. | ||
* | ||
* @param consumer An additional PTransform that should process the input PCollection. Its output | ||
* will be ignored. | ||
* @param <T> the type of the elements in the input {@code PCollection}. | ||
*/ | ||
public static <T> Tee<T> of(PTransform<PCollection<T>, ?> consumer) { | ||
return new Tee<>(consumer); | ||
} | ||
|
||
/** | ||
* Returns a new Tee PTransform that will apply an auxilary transform to the input as well as pass | ||
* it on. | ||
* | ||
* @param consumer An arbitrary {@link Consumer} that will be wrapped in a PTransform and applied | ||
* to the input. Its output will be ignored. | ||
* @param <T> the type of the elements in the input {@code PCollection}. | ||
*/ | ||
public static <T> Tee<T> of(Consumer<PCollection<T>> consumer) { | ||
return of( | ||
new PTransform<PCollection<T>, PCollectionTuple>() { | ||
@Override | ||
public PCollectionTuple expand(PCollection<T> input) { | ||
consumer.accept(input); | ||
return PCollectionTuple.empty(input.getPipeline()); | ||
} | ||
}); | ||
} | ||
|
||
private Tee(PTransform<PCollection<T>, ?> consumer) { | ||
this.consumer = consumer; | ||
} | ||
|
||
@Override | ||
public PCollection<T> expand(PCollection<T> input) { | ||
input.apply(consumer); | ||
return input; | ||
} | ||
|
||
@Override | ||
protected String getKindString() { | ||
return "Tee(" + consumer.getName() + ")"; | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,84 @@ | ||
/* | ||
* 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.sdk.transforms; | ||
|
||
import static org.hamcrest.MatcherAssert.assertThat; | ||
import static org.hamcrest.Matchers.containsInAnyOrder; | ||
|
||
import java.util.Arrays; | ||
import java.util.Collection; | ||
import java.util.List; | ||
import java.util.UUID; | ||
import org.apache.beam.sdk.testing.NeedsRunner; | ||
import org.apache.beam.sdk.testing.PAssert; | ||
import org.apache.beam.sdk.testing.TestPipeline; | ||
import org.apache.beam.sdk.values.PCollection; | ||
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.HashMultimap; | ||
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Multimap; | ||
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Multimaps; | ||
import org.junit.Rule; | ||
import org.junit.Test; | ||
import org.junit.experimental.categories.Category; | ||
import org.junit.runner.RunWith; | ||
import org.junit.runners.JUnit4; | ||
|
||
/** Tests for Tee. */ | ||
@RunWith(JUnit4.class) | ||
public class TeeTest { | ||
|
||
@Rule public final transient TestPipeline p = TestPipeline.create(); | ||
|
||
@Test | ||
@Category(NeedsRunner.class) | ||
public void testTee() { | ||
List<String> elements = Arrays.asList("a", "b", "c"); | ||
CollectToMemory<String> collector = new CollectToMemory<>(); | ||
PCollection<String> output = p.apply(Create.of(elements)).apply(Tee.of(collector)); | ||
|
||
PAssert.that(output).containsInAnyOrder(elements); | ||
p.run().waitUntilFinish(); | ||
|
||
// Here we assert that this "sink" had the correct side effects. | ||
assertThat(collector.get(), containsInAnyOrder(elements.toArray(new String[3]))); | ||
} | ||
|
||
private static class CollectToMemory<T> extends PTransform<PCollection<T>, PCollection<Void>> { | ||
|
||
private static final Multimap<UUID, Object> ALL_ELEMENTS = | ||
Multimaps.synchronizedMultimap(HashMultimap.<UUID, Object>create()); | ||
|
||
UUID uuid = UUID.randomUUID(); | ||
|
||
@Override | ||
public PCollection<Void> expand(PCollection<T> input) { | ||
return input.apply( | ||
ParDo.of( | ||
new DoFn<T, Void>() { | ||
@ProcessElement | ||
public void processElement(ProcessContext c) { | ||
ALL_ELEMENTS.put(uuid, c.element()); | ||
} | ||
})); | ||
} | ||
|
||
@SuppressWarnings("unchecked") | ||
public Collection<T> get() { | ||
return (Collection<T>) ALL_ELEMENTS.get(uuid); | ||
} | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -1143,6 +1143,60 @@ def model_multiple_pcollections_flatten(contents, output_path): | |
merged | beam.io.WriteToText(output_path) | ||
|
||
|
||
def model_multiple_pcollections_flatten_with(contents, output_path): | ||
"""Merging a PCollection with FlattenWith.""" | ||
some_hash_fn = lambda s: ord(s[0]) | ||
partition_fn = lambda element, partitions: some_hash_fn(element) % partitions | ||
import apache_beam as beam | ||
with TestPipeline() as pipeline: # Use TestPipeline for testing. | ||
|
||
# Partition into deciles | ||
partitioned = pipeline | beam.Create(contents) | beam.Partition( | ||
partition_fn, 3) | ||
pcoll1 = partitioned[0] | ||
pcoll2 = partitioned[1] | ||
pcoll3 = partitioned[2] | ||
SomeTransform = lambda: beam.Map(lambda x: x) | ||
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. Should we have other example for merging with a transform output? feel free to create a bug to add it. examples are just important as having the capability, so i think we should highlight these everywhere (beam playground, snippets, website docs, etc). Can be with follow up /starter bugs if you don't have time to do all that in one change. 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. Added this example, filed #32840 for follow-up. It would be good to think about how we could structure things to further reduce redundancy between these various forms of documentation. |
||
SomeOtherTransform = lambda: beam.Map(lambda x: x) | ||
|
||
# Flatten them back into 1 | ||
|
||
# A collection of PCollection objects can be represented simply | ||
# as a tuple (or list) of PCollections. | ||
# (The SDK for Python has no separate type to store multiple | ||
# PCollection objects, whether containing the same or different | ||
# types.) | ||
# [START model_multiple_pcollections_flatten_with] | ||
merged = ( | ||
pcoll1 | ||
| SomeTransform() | ||
| beam.FlattenWith(pcoll2, pcoll3) | ||
| SomeOtherTransform()) | ||
# [END model_multiple_pcollections_flatten_with] | ||
merged | beam.io.WriteToText(output_path) | ||
|
||
|
||
def model_multiple_pcollections_flatten_with_transform(contents, output_path): | ||
"""Merging output of PTransform with FlattenWith.""" | ||
some_hash_fn = lambda s: ord(s[0]) | ||
partition_fn = lambda element, partitions: some_hash_fn(element) % partitions | ||
import apache_beam as beam | ||
with TestPipeline() as pipeline: # Use TestPipeline for testing. | ||
|
||
pcoll = pipeline | beam.Create(contents) | ||
SomeTransform = lambda: beam.Map(lambda x: x) | ||
SomeOtherTransform = lambda: beam.Map(lambda x: x) | ||
|
||
# [START model_multiple_pcollections_flatten_with] | ||
merged = ( | ||
pcoll | ||
| SomeTransform() | ||
| beam.FlattenWith(beam.Create(['x', 'y', 'z'])) | ||
| SomeOtherTransform()) | ||
# [END model_multiple_pcollections_flatten_with] | ||
merged | beam.io.WriteToText(output_path) | ||
|
||
|
||
def model_multiple_pcollections_partition(contents, output_path): | ||
"""Splitting a PCollection with Partition.""" | ||
some_hash_fn = lambda s: ord(s[0]) | ||
|
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.
Merge.With might be a possible alternative name. but maybe it adds more confusion since we have a pre-existing
Flatten
already for a similar concept.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.
Yeah, I chose this name because it is literally syntactic sugar for the same primitive Flatten operation. (Personally, I'd prefer disjoint union, but that's probably to obscure let alone too late to change now...)