-
Notifications
You must be signed in to change notification settings - Fork 4.6k
update transform catalogue docs #38457
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
base: master
Are you sure you want to change the base?
Changes from all commits
9ff254e
05d4ef8
480cc3c
08efcd4
af1aa24
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,81 @@ | ||
| /* | ||
| * 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.examples; | ||
|
|
||
| import java.util.List; | ||
| import org.apache.beam.sdk.Pipeline; | ||
| import org.apache.beam.sdk.options.PipelineOptions; | ||
| import org.apache.beam.sdk.options.PipelineOptionsFactory; | ||
| import org.apache.beam.sdk.transforms.BatchElements; | ||
| import org.apache.beam.sdk.transforms.Create; | ||
| import org.apache.beam.sdk.transforms.DoFn; | ||
| import org.apache.beam.sdk.transforms.ParDo; | ||
| import org.apache.beam.sdk.values.PCollection; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
| // beam-playground: | ||
| // name: BatchElements | ||
| // description: Demonstration of BatchElements transform usage. | ||
| // multifile: false | ||
| // default_example: false | ||
| // context_line: 47 | ||
| // categories: | ||
| // - Core Transforms | ||
| // complexity: BASIC | ||
| // tags: | ||
| // - transforms | ||
| // - batch | ||
|
|
||
| public class BatchElementsExample { | ||
| public static void main(String[] args) { | ||
| PipelineOptions options = PipelineOptionsFactory.create(); | ||
|
|
||
| Pipeline pipeline = Pipeline.create(options); | ||
|
|
||
| // [START main_section] | ||
| // Create input | ||
|
|
||
| PCollection<String> inputs = | ||
| pipeline.apply(Create.of("apple", "strawberry", "orange", "peach", "cherry", "pear")); | ||
|
|
||
| // Create Batch Config | ||
| BatchElements.BatchConfig config = | ||
| BatchElements.BatchConfig.builder().withMinBatchSize(2).withMaxBatchSize(4).build(); | ||
| // Batch Elements | ||
| PCollection<List<String>> result = inputs.apply(BatchElements.withConfig(config)); | ||
| // [END main_section] | ||
| result.apply(ParDo.of(new LogOutput())); | ||
| pipeline.run(); | ||
| } | ||
|
|
||
| static class LogOutput extends DoFn<List<String>, String> { | ||
| private static final Logger LOG = LoggerFactory.getLogger(LogOutput.class); | ||
|
|
||
| @ProcessElement | ||
| public void processElement(ProcessContext c) throws Exception { | ||
| List<String> batch = c.element(); | ||
|
|
||
| LOG.info("Batch Contents: {}", batch); | ||
|
|
||
| for (String element : batch) { | ||
| c.output(element); | ||
| } | ||
| } | ||
|
Comment on lines
+71
to
+79
Contributor
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. In modern Beam, it is preferred to use parameter injection (e.g., public void processElement(@Element List<String> batch, OutputReceiver<String> receiver) {
LOG.info("Batch Contents: {}", batch);
for (String element : batch) {
receiver.output(element);
}
} |
||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,6 +17,8 @@ | |
| */ | ||
| package org.apache.beam.sdk.transforms; | ||
|
|
||
| import static java.util.Collections.singleton; | ||
|
|
||
| import java.io.Serializable; | ||
| import java.util.ArrayList; | ||
| import java.util.Comparator; | ||
|
|
@@ -27,6 +29,7 @@ | |
| import javax.annotation.Nullable; | ||
| import org.apache.beam.sdk.transforms.windowing.BoundedWindow; | ||
| import org.apache.beam.sdk.transforms.windowing.GlobalWindow; | ||
| import org.apache.beam.sdk.transforms.windowing.PaneInfo; | ||
| import org.apache.beam.sdk.values.PCollection; | ||
| import org.apache.beam.sdk.values.WindowingStrategy; | ||
|
|
||
|
|
@@ -568,7 +571,11 @@ public void processElement( | |
|
|
||
| try (BatchSizeEstimator.Stopwatch sw = estimator.recordTime(targetBatch.size)) { | ||
|
|
||
| receiver.outputWithTimestamp(targetBatch.elements, targetWindow.maxTimestamp()); | ||
| receiver.outputWindowedValue( | ||
| targetBatch.elements, | ||
| targetWindow.maxTimestamp(), | ||
| singleton(targetWindow), | ||
| PaneInfo.NO_FIRING); | ||
|
Comment on lines
+574
to
+578
Contributor
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. The
Contributor
Author
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. |
||
| } | ||
|
|
||
| batches.remove(targetWindow); | ||
|
|
||
| Original file line number | Diff line number | Diff line change | ||||||||||||||||
|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|---|
| @@ -0,0 +1,30 @@ | ||||||||||||||||||
| --- | ||||||||||||||||||
| title: "BatchElements" | ||||||||||||||||||
| --- | ||||||||||||||||||
| <!-- | ||||||||||||||||||
| 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. | ||||||||||||||||||
| --> | ||||||||||||||||||
|
|
||||||||||||||||||
| # BatchElements | ||||||||||||||||||
|
|
||||||||||||||||||
| BatchElements transform groups individual elements into batches before processing them downstream. | ||||||||||||||||||
| The transform takes a `PCollection<T>` as input and produces a `PCollection<List<T>>`, where each output element is a batch containing multiple input elements. | ||||||||||||||||||
| Batch sizes are chosen dynamically between the configured minimum and maximum values by measuring the execution time of downstream operations. | ||||||||||||||||||
|
|
||||||||||||||||||
| Batching is performed per window. Each emitted batch belongs to the same window as its input elements. | ||||||||||||||||||
|
Comment on lines
+18
to
+24
Contributor
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. Maybe include some motivation behind BatchElements? This is a good excerpt: beam/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/BatchElements.java Lines 36 to 43 in cf5e517
|
||||||||||||||||||
|
|
||||||||||||||||||
| ## Examples | ||||||||||||||||||
|
|
||||||||||||||||||
| {{< playground height="700px" >}} | ||||||||||||||||||
| {{< playground_snippet language="java" path="SDK_JAVA_BatchElements" show="main_section" >}} | ||||||||||||||||||
| {{< /playground >}} | ||||||||||||||||||
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.
It is recommended to use
PipelineOptionsFactory.fromArgs(args).withValidation().create()instead ofPipelineOptionsFactory.create(). This allows users to pass command-line arguments, such as--runner, to configure the pipeline execution when running the example.