├── .github
└── workflows
│ ├── java.yml
│ └── python.yml
├── .gitignore
├── CONTRIBUTING.md
├── Java
├── .gitignore
├── pom.xml
└── src
│ └── main
│ └── java
│ ├── advanced
│ ├── CustomWindows.java
│ ├── StatefulDoFn.java
│ └── TimerDoFn.java
│ ├── basics
│ ├── CombinerInterface.java
│ ├── ExpandPTransform.java
│ ├── ParDoInterface.java
│ ├── ParDoWithOutputs.java
│ ├── SideInputs.java
│ ├── coGroupByKey.java
│ └── groupByKey.java
│ ├── bigquery
│ ├── FailedRowsBQ.java
│ ├── NestedRowsBQ.java
│ ├── ReadQueryBQ.java
│ ├── ReadStorageAPIBQ.java
│ ├── ReadTableBQ.java
│ ├── ReadTableReferenceBQ.java
│ ├── RepeatedRowsBQ.java
│ ├── StreamingInsertsBQ.java
│ ├── StreamingLoadJobBQ.java
│ ├── WriteBQ.java
│ ├── WriteDynamicBQ.java
│ ├── WriteDynamicBQ2.java
│ ├── WritePartitionedClusteredTableBQ.java
│ ├── WriteWithAvroWriterBQ.java
│ └── WriteWithFormatBQ.java
│ ├── bigtable
│ ├── ReadBigTable.java
│ ├── ReadWithRowFilterBigTable.java
│ └── WriteBigTable.java
│ ├── cloudsql
│ ├── BQtoCloudSQL.java
│ ├── ReadCloudSQL.java
│ ├── ReadRowsCloudSQL.java
│ ├── WriteCloudSQL.java
│ └── WriteRowCloudSQL.java
│ ├── datastore
│ ├── ReadDatastore.java
│ └── WriteDatastore.java
│ ├── extra
│ ├── CustomMetrics.java
│ ├── FileSystemsDynamic.java
│ ├── JVMInitializer.java
│ ├── SSLCiphers.java
│ └── Template.java
│ ├── gcs
│ ├── MatchAllContinuouslyFileIO.java
│ ├── MatchAllFileIO.java
│ ├── MatchAllFileIOStreaming.java
│ ├── ReadGenericAvroIO.java
│ ├── ReadTextIO.java
│ ├── WriteAvroDynamic.java
│ ├── WriteCsvIO.java
│ ├── WriteDynamicFileIO.java
│ ├── WriteDynamicFileIOStreaming.java
│ ├── WriteJsonIO.java
│ ├── WriteStreamingTextIO.java
│ └── WriteTextIO.java
│ ├── jdbc
│ ├── ReadPartitionsJdbc.java
│ └── WriteJdbc.java
│ ├── kafka
│ ├── ReadKafka.java
│ ├── ReadWithBadRecordErrorHandlerKafka.java
│ ├── WriteDynamicKafka.java
│ ├── WriteKafka.java
│ └── WriteStreamingKafka.java
│ ├── minimal
│ ├── create.java
│ └── createWithParameters.java
│ ├── mongodb
│ ├── ReadMongoDB.java
│ └── WriteMongoDB.java
│ ├── parquet
│ ├── ReadParquet.java
│ └── WriteParquet.java
│ ├── pubsub
│ ├── ReadSubscriptionPubSub.java
│ ├── ReadTopicPubSub.java
│ ├── ReadWithAttributesPubSub.java
│ ├── ReadWithDeadLetterTopicPubSub.java
│ ├── WriteStringPubSub.java
│ └── WriteWithAttributesPubSub.java
│ ├── pubsublite
│ ├── ReadPubSubLite.java
│ └── WritePubSubLite.java
│ ├── spanner
│ ├── ReadQuerySpanner.java
│ ├── ReadTableSpanner.java
│ ├── ReadWithTransactionSpanner.java
│ ├── WriteGroupedSpanner.java
│ └── WriteSpanner.java
│ ├── sql
│ ├── BasicSQL.java
│ ├── BigQuerySQL.java
│ ├── JoinSQL.java
│ ├── JoinSQLWithGroups.java
│ ├── StreamingSQL.java
│ ├── StreamingSourceSQL.java
│ ├── UDFSQL.java
│ ├── WindowingSQL.java
│ └── ZetaSQL.java
│ ├── testingwindows
│ ├── AccumulatingFiredPanes.java
│ ├── AfterEachTrigger.java
│ ├── AfterFirstTrigger.java
│ ├── DiscardingFiredPanes.java
│ ├── ElementCountTrigger.java
│ ├── LateData.java
│ ├── README.md
│ └── WithLateAndEarlyFirings.java
│ ├── tfrecord
│ ├── ReadTFRecord.java
│ └── WriteTFRecord.java
│ └── windows
│ ├── FixedWindow.java
│ ├── GlobalWindow.java
│ ├── SessionWindow.java
│ └── SlidingWindow.java
├── LICENSE
├── Python
├── .flake8
├── .gitignore
├── .isort.cfg
├── .pre-commit-config.yaml
├── CONTRIBUTING.md
├── Makefile
├── advanced
│ ├── stateful_dofn.py
│ └── timer_dofn.py
├── basics
│ ├── builtin_combiners.py
│ ├── co_group_by_key.py
│ ├── co_group_by_key_with_dict.py
│ ├── combine_globally.py
│ ├── combine_interface.py
│ ├── combine_per_key.py
│ ├── expand_ptransform.py
│ ├── group_by_key.py
│ ├── map.py
│ ├── pardo.py
│ ├── pardo_interface.py
│ ├── pardo_with_output.py
│ ├── side_input_dictionary.py
│ ├── side_input_iterator.py
│ └── side_input_singleton.py
├── bigquery
│ ├── failed_rows_bigquery.py
│ ├── gcloud_logs_filter_with_dlq.py
│ ├── nested_bigquery.py
│ ├── read_all_bigquery.py
│ ├── read_query_bigquery.py
│ ├── read_table_bigquery.py
│ ├── read_table_ref_bigquery.py
│ ├── repeated_bigquery.py
│ ├── streaming_inserts_bigquery.py
│ ├── streaming_load_jobs_bigquery.py
│ ├── write_bigquery.py
│ ├── write_bigquery_dynamic.py
│ ├── write_bigquery_dynamic_side_input.py
│ └── write_partitioned_cluster_table_bigquery.py
├── bigtable
│ ├── bigtable_to_gcs_with_filter.py
│ ├── read_bigtable.py
│ └── write_bigtable.py
├── csv
│ ├── read_csv.py
│ └── write_csv.py
├── extra_examples
│ ├── custom_metrics.py
│ ├── file_system_dynamics.py
│ └── template.py
├── file
│ ├── match_files.py
│ ├── match_files_continuously.py
│ └── write_files.py
├── gcs
│ ├── match_continuously.py
│ ├── read_all_textio.py
│ ├── read_all_textio_streaming.py
│ ├── read_avro.py
│ ├── read_parquet.py
│ ├── read_textio.py
│ ├── write_avro.py
│ ├── write_parquet.py
│ └── write_textio.py
├── jdbc
│ ├── read_jdbc.py
│ └── write_jdbc.py
├── json
│ ├── read_json.py
│ └── write_json.py
├── kafka
│ ├── gcloud_logs_filter_with_dlq.py
│ ├── read_kafka.py
│ └── write_kafka.py
├── minimal
│ ├── create.py
│ ├── create_parameters.py
│ └── streaming.py
├── mongodb
│ ├── read_mongodb.py
│ └── write_mongodb.py
├── pubsub
│ ├── gcloud_logs_filter_with_dlq.py
│ ├── read_pubsub_multiple.py
│ ├── read_pubsub_subscription.py
│ ├── read_pubsub_topic.py
│ ├── read_pubsub_topic_attributes.py
│ ├── write_pubsub.py
│ └── write_pubsub_attributes.py
├── pyproject.toml
├── requirements.dev.txt
├── requirements.txt
├── spanner
│ ├── delete_spanner.py
│ ├── gcloud_logs_filter_with_dlq.py
│ ├── insert_or_update_spanner.py
│ ├── insert_spanner.py
│ ├── read_spanner.py
│ ├── replace_spanner.py
│ └── update_spanner.py
├── testing_windows
│ ├── README.md
│ ├── accumulating_fired_panes.py
│ ├── discarding_fired_panes.py
│ ├── element_count_trigger.py
│ ├── late_data.py
│ └── with_late_and_early_firings.py
├── tfrecord
│ ├── read_tfrecord.py
│ └── write_tfrecord.py
├── windows
│ ├── fixed_windows.py
│ ├── global_window.py
│ ├── session_window.py
│ └── sliding_window.py
└── yaml
│ ├── README.md
│ ├── bigquery_to_kafka.yaml
│ ├── kafka_to_bigquery.yaml
│ ├── kafka_to_json.yaml
│ └── test_csv_to_json.yaml
├── README.md
├── Scala
├── .gitignore
├── README.md
└── src
│ └── main
│ └── scala
│ ├── advanced
│ ├── ParDoInterface.scala
│ ├── StatefulDoFn.scala
│ └── TimerDoFn.scala
│ ├── basics
│ ├── ApplyTransform.scala
│ ├── CoGroupByKey.scala
│ ├── CombineGlobally.scala
│ ├── CombineGloballyInterface.scala
│ ├── CombinePerKey.scala
│ ├── CombinePerKeyInterface.scala
│ ├── CreateInteger.scala
│ ├── CreateIterator.scala
│ ├── Flatten.scala
│ ├── GroupByKey.scala
│ ├── Join.scala
│ ├── ListSideInput.scala
│ ├── MapSideInput.scala
│ ├── MultipleOutputs.scala
│ ├── PardoFns.scala
│ └── ScioFlatten.scala
│ ├── bigquery
│ ├── NestedRows.scala
│ ├── ReadQuery.scala
│ ├── ReadStorageAPI.scala
│ ├── ReadTable.scala
│ ├── ReadTypedTable.scala
│ ├── RepeatedRows.scala
│ ├── WriteBatch.scala
│ ├── WriteDynamicTable.scala
│ ├── WriteLoadJobs.scala
│ └── WriteStreamingInserts.scala
│ ├── extra
│ ├── ClassicTemplate.scala
│ └── CustomMetrics.scala
│ ├── gcs
│ ├── AvroInOut.scala
│ ├── ReadFiles.scala
│ ├── ReadText.scala
│ ├── WriteDynamic.scala
│ ├── WriteText.scala
│ └── WriteTextStreaming.scala
│ ├── minimal
│ └── minimal.scala
│ ├── pubsub
│ ├── ReadTopic.scala
│ ├── ReadWithAttributes.scala
│ ├── WriteTopic.scala
│ └── WriteWithAttributes.scala
│ └── windows
│ ├── FixedWindows.scala
│ ├── GlobalWindows.scala
│ ├── SessionWindows.scala
│ └── SlidingWindows.scala
└── prompts.md
/.github/workflows/java.yml:
--------------------------------------------------------------------------------
1 | # Copyright 2023 Google LLC
2 |
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 |
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 |
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # This workflow will build a Java project with Maven, and cache/restore any dependencies to improve the workflow execution time
16 | name: Java CI
17 |
18 | on: [push, pull_request, workflow_dispatch]
19 |
20 | jobs:
21 | build:
22 |
23 | runs-on: ubuntu-latest
24 |
25 | steps:
26 | - uses: actions/checkout@v3
27 | - name: Set up JDK 17
28 | uses: actions/setup-java@v3
29 | with:
30 | java-version: '17'
31 | distribution: 'temurin'
32 | cache: maven
33 | - name: Build with Maven
34 | working-directory: ./Java
35 | run: mvn -B package
36 |
--------------------------------------------------------------------------------
/.github/workflows/python.yml:
--------------------------------------------------------------------------------
1 | # Copyright 2023 Google LLC
2 |
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 |
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 |
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | name: Run basic tests with Python 3.8
16 |
17 | on: [push, pull_request, workflow_dispatch]
18 |
19 | jobs:
20 | tests:
21 | runs-on: ubuntu-latest
22 |
23 | steps:
24 | - uses: actions/checkout@v3
25 | - name: Set up Python 3.8
26 | uses: actions/setup-python@v4
27 | with:
28 | python-version: "3.8"
29 | - name: Init env
30 | run: |
31 | cd Python
32 | make init
33 | - name: Run format and lint
34 | run: |
35 | cd Python
36 | make format
37 | make lint
38 |
--------------------------------------------------------------------------------
/.gitignore:
--------------------------------------------------------------------------------
1 | .vscode
--------------------------------------------------------------------------------
/CONTRIBUTING.md:
--------------------------------------------------------------------------------
1 | # How to Contribute
2 |
3 | We'd love to accept your patches and contributions to this project. There are
4 | just a few small guidelines you need to follow.
5 |
6 | 1 - The pipelines should be as minimal as possible and self-contained.
7 |
8 | 2 - When possible, use public sources (public PubSub topic, GCS bucket, BQ
9 | table...) . If this is not possible, add a comment explaining the needed
10 | process.
11 |
12 | 3 - It's OK to add comments as clarifications.
13 |
14 | 4 - Use logging as a way to show the pipeline is correct, for example, if the
15 | pipeline reads from a source, add a step that handles the read element and log
16 | it. If the amount of logging is considerable, add an aggregation or filter step.
17 |
18 | 5 - No need to add dead letter paths to the pipeline, unless required by the use
19 | case.
20 |
21 | ## Contributor License Agreement
22 |
23 | Contributions to this project must be accompanied by a Contributor License
24 | Agreement (CLA). You (or your employer)
25 | retain the copyright to your contribution; this simply gives us permission to
26 | use and redistribute your contributions as part of the project. Head over to
27 | to see your current agreements on file or
28 | to sign a new one.
29 |
30 | You generally only need to submit a CLA once, so if you've already submitted one
31 | (even if it was for a different project), you probably don't need to do it
32 | again.
33 |
34 | ## Code Reviews
35 |
36 | All submissions, including submissions by project members, require review. We
37 | use GitHub pull requests for this purpose. Consult
38 | [GitHub Help](https://help.github.com/articles/about-pull-requests/) for more
39 | information on using pull requests.
40 |
41 | ## Community Guidelines
42 |
43 | This project follows
44 | [Google's Open Source Community Guidelines](https://opensource.google/conduct/).
45 |
--------------------------------------------------------------------------------
/Java/.gitignore:
--------------------------------------------------------------------------------
1 | # Target folder
2 | target/
3 |
4 | # Compiled class file
5 | *.class
6 |
7 | # Log file
8 | *.log
9 |
10 | # BlueJ files
11 | *.ctxt
12 |
13 | # Mobile Tools for Java (J2ME)
14 | .mtj.tmp/
15 |
16 | # Package Files #
17 | *.jar
18 | *.war
19 | *.nar
20 | *.ear
21 | *.zip
22 | *.tar.gz
23 | *.rar
24 |
25 | # virtual machine crash logs, see http://www.java.com/en/download/help/error_hotspot.xml
26 | hs_err_pid*
27 | replay_pid*
28 |
29 |
--------------------------------------------------------------------------------
/Java/src/main/java/bigquery/ReadTableBQ.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2022 Google LLC
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * https://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package bigquery;
18 | import com.google.api.services.bigquery.model.TableRow;
19 | import org.apache.beam.sdk.Pipeline;
20 | import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO;
21 | import org.apache.beam.sdk.options.Default;
22 | import org.apache.beam.sdk.options.Description;
23 | import org.apache.beam.sdk.options.PipelineOptions;
24 | import org.apache.beam.sdk.options.PipelineOptionsFactory;
25 | import org.apache.beam.sdk.transforms.DoFn;
26 | import org.apache.beam.sdk.transforms.ParDo;
27 | import org.slf4j.Logger;
28 | import org.slf4j.LoggerFactory;
29 |
30 | public class ReadTableBQ {
31 | private static final Logger LOG = LoggerFactory.getLogger(ReadTableBQ.class);
32 |
33 | // Parameter parser
34 | public interface ReadTableBQOptions extends PipelineOptions {
35 | @Description("Table to read")
36 | @Default.String("bigquery-public-data:austin_bikeshare.bikeshare_stations")
37 | String getTable();
38 |
39 | void setTable(String value);
40 | }
41 |
42 | public static void main(String[] args) {
43 | // Reference the extended class
44 | ReadTableBQOptions options = PipelineOptionsFactory.fromArgs(args).withValidation().as(ReadTableBQOptions.class);
45 |
46 | Pipeline p = Pipeline.create(options);
47 |
48 | p
49 | .apply(BigQueryIO.readTableRows().from(options.getTable()))
50 | .apply(ParDo.of(new DoFn() {
51 | @ProcessElement
52 | public void processElement(ProcessContext c) {
53 | c.output(c.element().toString());
54 | }
55 | })
56 | );
57 |
58 | p.run();
59 | }
60 |
61 | }
62 |
--------------------------------------------------------------------------------
/Java/src/main/java/extra/SSLCiphers.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2022 Google LLC
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * https://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package extra;
18 |
19 | import com.google.common.base.Joiner;
20 | import org.apache.beam.sdk.Pipeline;
21 | import org.apache.beam.sdk.options.PipelineOptions;
22 | import org.apache.beam.sdk.options.PipelineOptionsFactory;
23 | import org.apache.beam.sdk.transforms.Create;
24 | import org.apache.beam.sdk.transforms.DoFn;
25 | import org.apache.beam.sdk.transforms.ParDo;
26 | import org.slf4j.Logger;
27 | import org.slf4j.LoggerFactory;
28 |
29 | import javax.net.ssl.SSLServerSocketFactory;
30 |
31 | public class SSLCiphers {
32 | /*
33 | This pipeline lists as a log the Ciphers used by the workers
34 | */
35 |
36 | private static final Logger LOG = LoggerFactory.getLogger(SSLCiphers.class);
37 |
38 | public static void main(String[] args) {
39 |
40 | PipelineOptions options = PipelineOptionsFactory.fromArgs(args).withValidation().as(PipelineOptions.class);
41 |
42 | Pipeline p = Pipeline.create(options);
43 |
44 | p
45 | .apply("Create Element", Create.of(1))
46 | .apply("SSL logger", ParDo.of(new DoFn() {
47 | @ProcessElement
48 | public void processElement(ProcessContext c) {
49 | SSLServerSocketFactory fact = (SSLServerSocketFactory) SSLServerSocketFactory.getDefault();
50 | String ciphers = Joiner.on("\n").join(fact.getSupportedCipherSuites());
51 |
52 | LOG.info("Manual log of Ciphers: \n" + ciphers);
53 |
54 | c.output(ciphers);
55 | }
56 | }));
57 |
58 | p.run();
59 | }
60 | }
61 |
--------------------------------------------------------------------------------
/Java/src/main/java/gcs/MatchAllFileIO.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2022 Google LLC
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * https://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package gcs;
18 |
19 | import org.apache.beam.sdk.Pipeline;
20 | import org.apache.beam.sdk.io.FileIO;
21 | import org.apache.beam.sdk.io.TextIO;
22 | import org.apache.beam.sdk.options.PipelineOptions;
23 | import org.apache.beam.sdk.options.PipelineOptionsFactory;
24 | import org.apache.beam.sdk.transforms.Count;
25 | import org.apache.beam.sdk.transforms.Create;
26 | import org.apache.beam.sdk.transforms.DoFn;
27 | import org.apache.beam.sdk.transforms.ParDo;
28 | import org.slf4j.Logger;
29 | import org.slf4j.LoggerFactory;
30 |
31 | public class MatchAllFileIO {
32 |
33 | private static final Logger LOG = LoggerFactory.getLogger(MatchAllFileIO.class);
34 |
35 | public static void main(String[] args) {
36 | // Reference the extended class
37 | PipelineOptions options = PipelineOptionsFactory.fromArgs(args).withValidation().as(PipelineOptions.class);
38 |
39 | Pipeline p = Pipeline.create(options);
40 |
41 | p
42 | .apply(Create.of(
43 | "gs://apache-beam-samples/shakespeare/kinglear.txt",
44 | "gs://apache-beam-samples/shakespeare/macbeth.txt",
45 | "gs://apache-beam-samples/shakespeare/a*")) // Wildcards are also valid
46 | .apply(FileIO.matchAll()) // gets the files
47 | .apply(FileIO.readMatches()) // converts matches into readable files
48 | .apply(TextIO.readFiles()) // actual read
49 | .apply(Count.globally())
50 | .apply(ParDo.of(new DoFn() {
51 | @ProcessElement
52 | public void processElement(ProcessContext c) {
53 | LOG.info("Total lines " + c.element().toString());
54 | c.output(c.element());
55 | }
56 | }))
57 | ;
58 |
59 | p.run();
60 | }
61 | }
62 |
--------------------------------------------------------------------------------
/Java/src/main/java/gcs/ReadTextIO.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2022 Google LLC
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * https://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package gcs;
18 |
19 | import org.apache.beam.sdk.Pipeline;
20 | import org.apache.beam.sdk.io.TextIO;
21 | import org.apache.beam.sdk.options.Default;
22 | import org.apache.beam.sdk.options.Description;
23 | import org.apache.beam.sdk.options.PipelineOptions;
24 | import org.apache.beam.sdk.options.PipelineOptionsFactory;
25 | import org.apache.beam.sdk.transforms.Count;
26 | import org.apache.beam.sdk.transforms.DoFn;
27 | import org.apache.beam.sdk.transforms.ParDo;
28 | import org.slf4j.Logger;
29 | import org.slf4j.LoggerFactory;
30 |
31 | public class ReadTextIO {
32 |
33 | private static final Logger LOG = LoggerFactory.getLogger(ReadTextIO.class);
34 |
35 |
36 | public interface ReadTextIOOptions extends PipelineOptions {
37 | @Description("Path of the file to read from")
38 | @Default.String("gs://apache-beam-samples/shakespeare/kinglear.txt")
39 | String getInput();
40 |
41 | void setInput(String value);
42 | }
43 |
44 | public static void main(String[] args) {
45 | ReadTextIOOptions options = PipelineOptionsFactory.fromArgs(args).withValidation().as(ReadTextIOOptions.class);
46 |
47 | Pipeline p = Pipeline.create(options);
48 |
49 | p
50 | .apply(TextIO.read().from(options.getInput()))
51 | .apply(Count.globally())
52 | .apply(ParDo.of(new DoFn() {
53 | @ProcessElement
54 | public void processElement(ProcessContext c) {
55 | LOG.info("Total lines " + c.element().toString());
56 | c.output(c.element());
57 | }
58 | }));
59 |
60 | p.run();
61 |
62 |
63 | }
64 | }
65 |
--------------------------------------------------------------------------------
/Java/src/main/java/minimal/create.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2022 Google LLC
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * https://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package minimal;
18 |
19 | import org.apache.beam.sdk.Pipeline;
20 | import org.apache.beam.sdk.options.PipelineOptions;
21 | import org.apache.beam.sdk.options.PipelineOptionsFactory;
22 | import org.apache.beam.sdk.transforms.Create;
23 | import org.slf4j.Logger;
24 | import org.slf4j.LoggerFactory;
25 |
26 | public class create {
27 |
28 | private static final Logger LOG = LoggerFactory.getLogger(create.class);
29 |
30 | public static void main(String[] args) {
31 |
32 | PipelineOptions options = PipelineOptionsFactory.fromArgs(args).withValidation().as(PipelineOptions.class);
33 |
34 | Pipeline p = Pipeline.create(options);
35 |
36 | p
37 | .apply("Create Elements", Create.of(1, 2, 3, 4, 5));
38 |
39 | p.run();
40 | }
41 | }
42 |
43 |
--------------------------------------------------------------------------------
/Java/src/main/java/minimal/createWithParameters.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2022 Google LLC
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * https://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package minimal;
18 |
19 | import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
20 | import org.apache.beam.sdk.Pipeline;
21 | import org.apache.beam.sdk.options.Default;
22 | import org.apache.beam.sdk.options.Description;
23 | import org.apache.beam.sdk.options.PipelineOptionsFactory;
24 | import org.apache.beam.sdk.transforms.Create;
25 | import org.slf4j.Logger;
26 | import org.slf4j.LoggerFactory;
27 |
28 | public class createWithParameters {
29 |
30 | private static final Logger LOG = LoggerFactory.getLogger(create.class);
31 |
32 | // Parameter parser
33 | public interface MyOptions extends DataflowPipelineOptions {
34 | @Description("Extra element")
35 | @Default.Integer(1729)
36 | Integer getNumber();
37 |
38 | void setNumber(Integer value);
39 | }
40 |
41 | public static void main(String[] args) {
42 | // Reference the extended class
43 | MyOptions options = PipelineOptionsFactory.fromArgs(args).withValidation().as(MyOptions.class);
44 |
45 | Pipeline p = Pipeline.create(options);
46 |
47 | p.
48 | apply("Create Elements", Create.of(1, 2, 3, 4, 5,
49 | options.getNumber())); // Element accessible via options
50 |
51 | p.run();
52 | }
53 | }
54 |
--------------------------------------------------------------------------------
/Java/src/main/java/pubsub/ReadSubscriptionPubSub.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2022 Google LLC
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * https://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package pubsub;
18 | import org.apache.beam.sdk.Pipeline;
19 | import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO;
20 | import org.apache.beam.sdk.options.Description;
21 | import org.apache.beam.sdk.options.PipelineOptions;
22 | import org.apache.beam.sdk.options.PipelineOptionsFactory;
23 | import org.apache.beam.sdk.transforms.DoFn;
24 | import org.apache.beam.sdk.transforms.ParDo;
25 | import org.slf4j.Logger;
26 | import org.slf4j.LoggerFactory;
27 |
28 | public class ReadSubscriptionPubSub {
29 |
30 | private static final Logger LOG = LoggerFactory.getLogger(ReadSubscriptionPubSub.class);
31 |
32 | public interface ReadSubscriptionPubSubOptions extends PipelineOptions {
33 | @Description("Subscription to read from")
34 | String getSubscription();
35 |
36 | void setSubscription(String value);
37 | }
38 |
39 | public static void main(String[] args) {
40 | ReadSubscriptionPubSubOptions options = PipelineOptionsFactory.fromArgs(args).withValidation().as(ReadSubscriptionPubSubOptions.class);
41 |
42 | Pipeline p = Pipeline.create(options);
43 |
44 | p
45 | .apply("ReadFromPubSub", PubsubIO.readStrings().fromSubscription(options.getSubscription()))
46 | // Next ParDo is added so pipeline doesn't finish
47 | .apply("ParDo", ParDo.of(new DoFn() {
48 | @ProcessElement
49 | public void processElement(ProcessContext c) {
50 | c.output(c.element());
51 | }
52 | })
53 | );
54 |
55 | p.run();
56 | }
57 | }
58 |
--------------------------------------------------------------------------------
/Java/src/main/java/pubsub/ReadTopicPubSub.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2022 Google LLC
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * https://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package pubsub;
18 | import org.apache.beam.sdk.Pipeline;
19 | import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO;
20 | import org.apache.beam.sdk.options.Default;
21 | import org.apache.beam.sdk.options.Description;
22 | import org.apache.beam.sdk.options.PipelineOptions;
23 | import org.apache.beam.sdk.options.PipelineOptionsFactory;
24 | import org.apache.beam.sdk.transforms.DoFn;
25 | import org.apache.beam.sdk.transforms.ParDo;
26 | import org.slf4j.Logger;
27 | import org.slf4j.LoggerFactory;
28 |
29 |
30 | public class ReadTopicPubSub {
31 |
32 | private static final Logger LOG = LoggerFactory.getLogger(ReadTopicPubSub.class);
33 |
34 |
35 | public interface ReadTopicPubSubOptions extends PipelineOptions {
36 | @Description("Topic to read from")
37 | @Default.String("projects/pubsub-public-data/topics/taxirides-realtime")
38 | String getTopic();
39 |
40 | void setTopic(String value);
41 | }
42 |
43 | public static void main(String[] args) {
44 | ReadTopicPubSubOptions options = PipelineOptionsFactory.fromArgs(args).withValidation().as(ReadTopicPubSubOptions.class);
45 |
46 | Pipeline p = Pipeline.create(options);
47 |
48 | p
49 | // Reading from a Topic creates a subscription to it automatically
50 | .apply("ReadFromPubSub", PubsubIO.readStrings().fromTopic(options.getTopic()))
51 | // Next ParDo is added so pipeline doesn't finish
52 | .apply("ParDo", ParDo.of(new DoFn() {
53 | @ProcessElement
54 | public void processElement(ProcessContext c) {
55 | c.output(c.element());
56 | }
57 | })
58 | );
59 |
60 | p.run();
61 | }
62 | }
63 |
--------------------------------------------------------------------------------
/Java/src/main/java/pubsub/WriteStringPubSub.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2022 Google LLC
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * https://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package pubsub;
18 | import org.apache.beam.sdk.Pipeline;
19 | import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO;
20 | import org.apache.beam.sdk.options.Description;
21 | import org.apache.beam.sdk.options.PipelineOptions;
22 | import org.apache.beam.sdk.options.PipelineOptionsFactory;
23 | import org.apache.beam.sdk.transforms.Create;
24 | import org.slf4j.Logger;
25 | import org.slf4j.LoggerFactory;
26 |
27 | import java.util.Arrays;
28 | import java.util.List;
29 |
30 | public class WriteStringPubSub {
31 |
32 | private static final Logger LOG = LoggerFactory.getLogger(WriteStringPubSub.class);
33 |
34 | public interface WriteStringPubSubOptions extends PipelineOptions {
35 | @Description("Topic to write to")
36 | String getTopic();
37 |
38 | void setTopic(String value);
39 | }
40 |
41 | public static void main(String[] args) {
42 |
43 | WriteStringPubSubOptions options = PipelineOptionsFactory.fromArgs(args).withValidation().as(WriteStringPubSubOptions.class);
44 |
45 | Pipeline p = Pipeline.create(options);
46 |
47 | final List elements = Arrays.asList(
48 | "Message",
49 | "Another message",
50 | "Yet another message"
51 | );
52 |
53 | p
54 | .apply(Create.of(elements))
55 | .apply(PubsubIO.writeStrings().to(options.getTopic())); // Writing in Streaming works the same
56 |
57 | p.run();
58 | }
59 |
60 | }
61 |
--------------------------------------------------------------------------------
/Java/src/main/java/testingwindows/README.md:
--------------------------------------------------------------------------------
1 | ## Notes
2 |
3 | In order for the windows to take effect, you need to have aggregations or `StatefulDoFns`.
4 |
5 | 1 - Pipelines need to be ran locally. For example:
6 |
7 | ```
8 | mvn compile -e exec:java -Dexec.mainClass=testingWindows.ElementCountTrigger
9 | ```
10 |
11 | 2 - At the end of each file there's an explanation of the output
--------------------------------------------------------------------------------
/Python/.flake8:
--------------------------------------------------------------------------------
1 | [flake8]
2 | max-line-length = 80
3 | max-complexity = 40
4 | ignore =
5 | E203
6 | W503
7 | F841
8 | exclude =
9 | .eggs
10 | .git
11 | .tox
12 | __pycache__
13 | build
14 | dist
15 | venv
--------------------------------------------------------------------------------
/Python/.isort.cfg:
--------------------------------------------------------------------------------
1 | # Copyright 2023 Google LLC
2 |
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 |
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 |
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | [settings]
16 | sections=FUTURE,STDLIB,THIRDPARTY,FIRSTPARTY,LOCALFOLDER
17 | import_heading_stdlib=standard libraries
18 | import_heading_thirdparty=third party libraries
19 | include_trailing_comma=True
20 | indent=' '
21 | dedup_headings=True
22 | line_length=80
23 | multi_line_output=3
24 | skip=./venv/,./venv-docs/,./.git/
--------------------------------------------------------------------------------
/Python/.pre-commit-config.yaml:
--------------------------------------------------------------------------------
1 | # Copyright 2023 Google LLC
2 |
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 |
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 |
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | exclude: ^docs/notebooks/
16 | repos:
17 | - repo: https://github.com/ambv/black
18 | rev: 23.11.0
19 | hooks:
20 | - id: black
21 | args: ["--config=Python/pyproject.toml", "--check", "--diff"]
22 | - repo: https://github.com/pycqa/flake8
23 | rev: 6.1.0
24 | hooks:
25 | - id: flake8
26 | args: ["--config=Python/.flake8"]
27 | - repo: https://github.com/timothycrosley/isort
28 | rev: 5.12.0
29 | hooks:
30 | - id: isort
--------------------------------------------------------------------------------
/Python/CONTRIBUTING.md:
--------------------------------------------------------------------------------
1 | # How to Contribute Python Examples
2 |
3 | It is encouraged to install `make` to create your local development environment.
4 |
5 | 1. Create the local Python environment:
6 | ```bash
7 | make init
8 | ```
9 | 2. Use `source venv/bin/activate` to activate venv
10 | 3. Clean up the local enviroment:
11 | ```bash
12 | make clean
13 | ```
14 | 4. Format the Python code:
15 | ```bash
16 | make format
17 | ```
18 | 5. Run the Python code linter:
19 | ```bash
20 | make lint
21 | ```
22 |
--------------------------------------------------------------------------------
/Python/Makefile:
--------------------------------------------------------------------------------
1 | # Copyright 2023 Google LLC
2 |
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 |
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 |
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | SILENT:
16 | .PHONY:
17 | .DEFAULT_GOAL := help
18 |
19 | define PRINT_HELP_PYSCRIPT
20 | import re, sys # isort:skip
21 |
22 | matches = []
23 | for line in sys.stdin:
24 | match = re.match(r'^([a-zA-Z_-]+):.*?## (.*)$$', line)
25 | if match:
26 | matches.append(match.groups())
27 |
28 | for target, help in sorted(matches):
29 | print(" %-25s %s" % (target, help))
30 | endef
31 | export PRINT_HELP_PYSCRIPT
32 |
33 | PYTHON = python$(PYTHON_VERSION)
34 |
35 | ifndef TF_MODEL_URI
36 | MODEL_ENV := "TORCH"
37 | else
38 | MODEL_ENV := "TF"
39 | endif
40 |
41 | help: ## Print this help
42 | @echo
43 | @echo " make targets:"
44 | @echo
45 | @$(PYTHON) -c "$$PRINT_HELP_PYSCRIPT" < $(MAKEFILE_LIST)
46 |
47 | init-venv: ## Create virtual environment in venv folder
48 | @$(PYTHON) -m venv venv
49 |
50 | init: init-venv ## Init virtual environment
51 | @./venv/bin/python3 -m pip install -U pip
52 | @./venv/bin/python3 -m pip install -r requirements.txt
53 | @./venv/bin/python3 -m pip install -r requirements.dev.txt
54 | @./venv/bin/python3 -m pre_commit install --install-hooks --overwrite
55 | @echo "use 'source venv/bin/activate' to activate venv "
56 |
57 | format: ## Run formatter on source code
58 | @./venv/bin/python3 -m black --config=pyproject.toml .
59 |
60 | lint: ## Run linter on source code
61 | @./venv/bin/python3 -m black --config=pyproject.toml --check .
62 | @./venv/bin/python3 -m flake8 --config=.flake8 .
63 |
64 | clean-lite: ## Remove pycache files, pytest files, etc
65 | @rm -rf build dist .cache .coverage .coverage.* *.egg-info
66 | @find . -name .coverage | xargs rm -rf
67 | @find . -name .pytest_cache | xargs rm -rf
68 | @find . -name .tox | xargs rm -rf
69 | @find . -name __pycache__ | xargs rm -rf
70 | @find . -name *.egg-info | xargs rm -rf
71 |
72 | clean: clean-lite ## Remove virtual environment, downloaded models, etc
73 | @rm -rf venv
74 | @echo "run 'make init'"
75 |
--------------------------------------------------------------------------------
/Python/basics/builtin_combiners.py:
--------------------------------------------------------------------------------
1 | # Copyright 2022 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import Create, Map
21 | from apache_beam.transforms.combiners import Count, Mean, Top
22 |
23 | """
24 |
25 | Full list of combiners
26 | https://beam.apache.org/releases/pydoc/current/apache_beam.transforms.combiners.html
27 | """
28 |
29 |
30 | def run(argv=None):
31 | elements = [
32 | ("Mammal", "Dog"),
33 | ("Mammal", "Cat"),
34 | ("Fish", "Salmon"),
35 | ("Amphibian", "Snake"),
36 | ("Bird", "Eagle"),
37 | ("Bird", "Owl"),
38 | ("Mammal", "Algo"),
39 | ]
40 |
41 | with beam.Pipeline() as p:
42 | # Counters Globally and PerKey
43 | input_pcol = p | Create(elements)
44 | element_count_total = (
45 | input_pcol
46 | | "Global Count" >> Count.Globally()
47 | | "Global Log" >> Map(logging.info)
48 | )
49 | element_count_grouped = (
50 | input_pcol
51 | | "Count PerKey" >> Count.PerKey()
52 | | "PerKey Log" >> Map(logging.info)
53 | )
54 | # Mean and Top
55 | numbers = p | "Create Numbers" >> Create(range(100))
56 |
57 | mean = numbers | Mean.Globally() | "Mean Log" >> Map(logging.info)
58 |
59 | top = numbers | Top.Of(5) | "Top Log" >> Map(logging.info)
60 |
61 |
62 | if __name__ == "__main__":
63 | logging.getLogger().setLevel(logging.INFO)
64 | run()
65 |
--------------------------------------------------------------------------------
/Python/basics/co_group_by_key.py:
--------------------------------------------------------------------------------
1 | # Copyright 2022 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import CoGroupByKey, Create, Map
21 |
22 |
23 | def run(argv=None):
24 | # Define the input key-value pairs.
25 | jobs = [
26 | ("Anna", "SWE"),
27 | ("Kim", "Data Engineer"),
28 | ("Kim", "Data Scientist"),
29 | ("Robert", "Artist"),
30 | ("Sophia", "CEO"),
31 | ]
32 | hobbies = [
33 | ("Anna", "Painting"),
34 | ("Kim", "Football"),
35 | ("Kim", "Gardening"),
36 | ("Robert", "Swimming"),
37 | ("Sophia", "Mathematics"),
38 | ("Sophia", "Tennis"),
39 | ]
40 |
41 | # Create two PCollections with beam.Create().
42 | # Use beam.CoGroupByKey() to group elements from these two PCollections
43 | # by their key.
44 | with beam.Pipeline() as p:
45 | jobs_pcol = p | "Create Jobs" >> Create(jobs)
46 | hobbies_pcol = p | "Create Hobbies" >> Create(hobbies)
47 | output = (
48 | (jobs_pcol, hobbies_pcol)
49 | | "CoGroupByKey" >> CoGroupByKey()
50 | | "Log" >> Map(logging.info)
51 | )
52 |
53 |
54 | if __name__ == "__main__":
55 | logging.getLogger().setLevel(logging.INFO)
56 | run()
57 |
--------------------------------------------------------------------------------
/Python/basics/co_group_by_key_with_dict.py:
--------------------------------------------------------------------------------
1 | # Copyright 2022 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import CoGroupByKey, Create, Map
21 |
22 |
23 | def run(argv=None):
24 | jobs = [
25 | ("Anna", "SWE"),
26 | ("Kim", "Data Engineer"),
27 | ("Kim", "Data Scientist"),
28 | ("Robert", "Artist"),
29 | ("Sophia", "CEO"),
30 | ]
31 | hobbies = [
32 | ("Anna", "Painting"),
33 | ("Kim", "Football"),
34 | ("Kim", "Gardening"),
35 | ("Robert", "Swimming"),
36 | ("Sophia", "Mathematics"),
37 | ("Sophia", "Tennis"),
38 | ]
39 |
40 | with beam.Pipeline() as p:
41 | jobs_pcol = p | "Create Jobs" >> Create(jobs)
42 | hobbies_pcol = p | "Create Hobbies" >> Create(hobbies)
43 | output = (
44 | {"Jobs": jobs_pcol, "Hobbies": hobbies_pcol}
45 | | CoGroupByKey()
46 | | "Log" >> Map(logging.info)
47 | )
48 |
49 |
50 | if __name__ == "__main__":
51 | logging.getLogger().setLevel(logging.INFO)
52 | run()
53 |
--------------------------------------------------------------------------------
/Python/basics/combine_globally.py:
--------------------------------------------------------------------------------
1 | # Copyright 2022 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import CombineGlobally, Create, Map
21 |
22 |
23 | def run(argv=None):
24 | elements = [
25 | "Lorem ipsum dolor sit amet. Consectetur adipiscing elit",
26 | "Sed eu velit nec sem vulputate loborti",
27 | "In lobortis augue vitae sagittis molestie. Mauris volutpat tortor non purus elementum", # noqa:E501
28 | "Ut blandit massa et risus sollicitudin auctor",
29 | ]
30 |
31 | with beam.Pipeline() as p:
32 | output = (
33 | p
34 | | Create(elements)
35 | | "Join" >> CombineGlobally(lambda x: ". ".join(x))
36 | | "Log" >> Map(logging.info)
37 | )
38 |
39 |
40 | if __name__ == "__main__":
41 | logging.getLogger().setLevel(logging.INFO)
42 | run()
43 |
--------------------------------------------------------------------------------
/Python/basics/combine_interface.py:
--------------------------------------------------------------------------------
1 | # Copyright 2022 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import CombineFn, CombineGlobally, Create, Map
21 |
22 |
23 | class AverageFn(CombineFn):
24 | """
25 | This is the same as the Mean PTransform, but it's used as an example
26 | to show the combiner interface
27 | """
28 |
29 | def create_accumulator(self):
30 | # create and initialise accumulator for sum and count
31 | initial_sum = 0
32 | initial_count = 0
33 | return initial_sum, initial_count
34 |
35 | def add_input(self, accumulator, element):
36 | # accumulates each element from input in accumulator
37 | new_total = accumulator[0] + element
38 | new_count = accumulator[1] + 1
39 | return new_total, new_count
40 |
41 | def merge_accumulators(self, accumulators):
42 | # Multiple accumulators could be processed in parallel,
43 | # this function merges them
44 | sums = [accumulator[0] for accumulator in accumulators]
45 | counts = [accumulator[1] for accumulator in accumulators]
46 | return sum(sums), sum(counts)
47 |
48 | def extract_output(self, accumulator):
49 | # calculations before final output
50 | return accumulator[0] / accumulator[1]
51 |
52 |
53 | def run(args=None):
54 | elements = range(100)
55 |
56 | with beam.Pipeline() as p:
57 | output = (
58 | p
59 | | Create(elements)
60 | | "Global Average" >> CombineGlobally(AverageFn())
61 | | "Log" >> Map(logging.info)
62 | )
63 |
64 |
65 | if __name__ == "__main__":
66 | logging.getLogger().setLevel(logging.INFO)
67 | run()
68 |
--------------------------------------------------------------------------------
/Python/basics/combine_per_key.py:
--------------------------------------------------------------------------------
1 | # Copyright 2022 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import CombinePerKey, Create, Map
21 |
22 |
23 | def run(argv=None):
24 | elements = [
25 | (
26 | "Latin",
27 | "Lorem ipsum dolor sit amet. Consectetur adipiscing elit. Sed eu velit nec sem vulputate loborti", # noqa:E501
28 | ),
29 | (
30 | "Latin",
31 | "In lobortis augue vitae sagittis molestie. Mauris volutpat tortor non purus elementum", # noqa:E501
32 | ),
33 | ("English", "From fairest creatures we desire increase"),
34 | ("English", "That thereby beauty's rose might never die"),
35 | ("English", "But as the riper should by time decease"),
36 | (
37 | "Spanish",
38 | "En un lugar de la Mancha, de cuyo nombre no quiero acordarme, no ha mucho", # noqa:E501
39 | ),
40 | (
41 | "Spanish",
42 | "tiempo que vivía un hidalgo de los de lanza en astillero, adarga antigua", # noqa:E501
43 | ),
44 | ]
45 |
46 | with beam.Pipeline() as p:
47 | output = (
48 | p
49 | | Create(elements)
50 | | "Join per key with full stop"
51 | >> CombinePerKey(lambda x: ". ".join(x))
52 | | "Log" >> Map(logging.info)
53 | )
54 |
55 |
56 | if __name__ == "__main__":
57 | logging.getLogger().setLevel(logging.INFO)
58 | run()
59 |
--------------------------------------------------------------------------------
/Python/basics/expand_ptransform.py:
--------------------------------------------------------------------------------
1 | # Copyright 2022 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 | import re
18 |
19 | # third party libraries
20 | import apache_beam as beam
21 | from apache_beam import CombinePerKey, Create, FlatMap, Map, PTransform
22 | from apache_beam.options.pipeline_options import PipelineOptions
23 |
24 |
25 | def run(argv=None):
26 | options = PipelineOptions()
27 |
28 | class WordCount(PTransform):
29 | def expand(self, pcoll):
30 | count = (
31 | pcoll
32 | | "Split" >> FlatMap(self._split_words)
33 | | "Count" >> CombinePerKey(sum)
34 | )
35 | return count
36 |
37 | def _split_words(self, text):
38 | words = re.findall(r"[\w\']+", text.strip(), re.UNICODE)
39 | return [(x, 1) for x in words]
40 |
41 | with beam.Pipeline(options=options) as p:
42 | elements = [
43 | "From fairest creatures we desire increase",
44 | "That thereby beauty’s rose might never die",
45 | "But as the riper should by time decease",
46 | "His tender heir might bear his memory",
47 | "But thou, contracted to thine own bright eyes",
48 | "Feed’st thy light’st flame with self-substantial fuel",
49 | "Making a famine where abundance lies",
50 | "Thyself thy foe, to thy sweet self too cruel",
51 | "Thou that art now the world’s fresh ornament",
52 | "And only herald to the gaudy spring",
53 | "Within thine own bud buriest thy content",
54 | "And, tender churl, makest waste in niggarding",
55 | "Pity the world, or else this glutton be",
56 | "To eat the world’s due, by the grave and thee",
57 | ]
58 |
59 | (p | Create(elements) | WordCount() | Map(logging.info))
60 |
61 |
62 | if __name__ == "__main__":
63 | logging.getLogger().setLevel(logging.INFO)
64 | run()
65 |
--------------------------------------------------------------------------------
/Python/basics/group_by_key.py:
--------------------------------------------------------------------------------
1 | # Copyright 2022 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import Create, GroupByKey, Map
21 |
22 |
23 | def run(argv=None):
24 | # Define the input key-value pairs
25 | elements = [
26 | ("Mammal", "Dog"),
27 | ("Mammal", "Cat"),
28 | ("Fish", "Salmon"),
29 | ("Amphibian", "Snake"),
30 | ("Bird", "Eagle"),
31 | ("Bird", "Owl"),
32 | ("Mammal", "Algo"),
33 | ]
34 |
35 | # Create a PCollection of elements defined above with beam.Create().
36 | # Use beam.GroupByKey() to group elements by their key.
37 | with beam.Pipeline() as p:
38 | output = (
39 | p
40 | | "Create Elements" >> Create(elements)
41 | | "Group Elements" >> GroupByKey()
42 | | "Log" >> Map(logging.info)
43 | )
44 |
45 |
46 | if __name__ == "__main__":
47 | logging.getLogger().setLevel(logging.INFO)
48 | run()
49 |
--------------------------------------------------------------------------------
/Python/basics/map.py:
--------------------------------------------------------------------------------
1 | # Copyright 2022 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import Create, Map
21 |
22 |
23 | def double_number(x):
24 | y = x * 2
25 | logging.info("double value %f" % y)
26 | return y
27 |
28 |
29 | def run(argv=None):
30 | n = 10
31 | with beam.Pipeline() as p:
32 | # Map can be used with lambda or defined function.
33 | output = (
34 | p
35 | | "Input" >> Create(range(n))
36 | | "Doubles" >> Map(double_number)
37 | | "Squares" >> Map(lambda x: x**2)
38 | | "Lambda" >> Map(logging.info)
39 | )
40 |
41 |
42 | if __name__ == "__main__":
43 | logging.getLogger().setLevel(logging.INFO)
44 | run()
45 |
--------------------------------------------------------------------------------
/Python/basics/pardo.py:
--------------------------------------------------------------------------------
1 | # Copyright 2022 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import Create, DoFn, Map, ParDo
21 |
22 |
23 | # ParDo allows 1 to Many outputs while Map only allows 1 to 1 outputs.
24 | class SplitFn(DoFn):
25 | def process(self, element):
26 | return element.split()
27 |
28 |
29 | def run(argv=None):
30 | elements = [
31 | "Lorem ipsum dolor sit amet. Consectetur adipiscing elit",
32 | "Sed eu velit nec sem vulputate loborti",
33 | "In lobortis augue vitae sagittis molestie. Mauris volutpat tortor non ", # noqa:E501
34 | "Ut blandit massa et risus sollicitudin auctor",
35 | ]
36 |
37 | with beam.Pipeline() as p:
38 | output = (
39 | p
40 | | Create(elements)
41 | | "Split Output" >> ParDo(SplitFn())
42 | | "Log" >> Map(logging.info)
43 | )
44 |
45 |
46 | if __name__ == "__main__":
47 | logging.getLogger().setLevel(logging.INFO)
48 | run()
49 |
--------------------------------------------------------------------------------
/Python/basics/pardo_interface.py:
--------------------------------------------------------------------------------
1 | # Copyright 2022 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import Create, DoFn, Map, ParDo, window
21 |
22 |
23 | class DoFnMethods(DoFn):
24 | def __init__(self):
25 | logging.debug("__init__")
26 | self.window = window.GlobalWindow()
27 | self.list = []
28 |
29 | def setup(self):
30 | logging.debug("setup")
31 |
32 | def start_bundle(self):
33 | logging.debug("start_bundle")
34 | self.list = []
35 |
36 | def process(self, element, window=DoFn.WindowParam):
37 | self.list.append(element)
38 | yield f"* process: {element}"
39 |
40 | def finish_bundle(self):
41 | # third party libraries
42 | from apache_beam.utils.windowed_value import WindowedValue
43 |
44 | # yielded elements from finish_bundle have to be type WindowedValue.
45 | yield WindowedValue(
46 | value=f"* finish_bundle: {self.list}",
47 | timestamp=0,
48 | windows=[self.window],
49 | )
50 |
51 | def teardown(self):
52 | logging.debug("teardown")
53 |
54 |
55 | def run(argv=None):
56 | n = 20
57 |
58 | with beam.Pipeline() as p:
59 | output = (
60 | p
61 | | Create(range(n))
62 | | "DoFn Methods" >> ParDo(DoFnMethods())
63 | | "Log" >> Map(logging.info)
64 | )
65 |
66 |
67 | if __name__ == "__main__":
68 | logging.getLogger().setLevel(logging.INFO)
69 | run()
70 |
--------------------------------------------------------------------------------
/Python/basics/pardo_with_output.py:
--------------------------------------------------------------------------------
1 | # Copyright 2022 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import Create, DoFn, Map, ParDo, pvalue
21 | from apache_beam.options.pipeline_options import PipelineOptions
22 |
23 |
24 | class SplitFn(DoFn):
25 | def process(self, element):
26 | # Generate 3 PCollections from the input:
27 | # 1) Even elements, with the 'even' tag
28 | # 2) Odd elements, with the 'odd' tag
29 | # 3) All elements emitted as the main untagged output
30 | if element % 2 == 0:
31 | yield pvalue.TaggedOutput("even", element)
32 | else:
33 | yield pvalue.TaggedOutput("odd", element)
34 | yield element
35 |
36 |
37 | def run(argv=None):
38 | n = 20
39 | options = PipelineOptions(save_main_session=True)
40 | with beam.Pipeline(options=options) as p:
41 | output = (
42 | p
43 | | Create(range(n))
44 | | "Split Output" >> ParDo(SplitFn()).with_outputs("even", "odd")
45 | )
46 |
47 | # Log each element of both tagged PCollections
48 | # and the main untagged PCollection
49 | odd = output.odd | "odd log" >> Map(
50 | lambda x: logging.info("odds %d" % x)
51 | )
52 | even = output.even | "even log" >> Map(
53 | lambda x: logging.info("evens %d" % x)
54 | )
55 | all_output = output[None] | "Log" >> Map(
56 | lambda x: logging.info("all %d" % x)
57 | )
58 |
59 |
60 | if __name__ == "__main__":
61 | logging.getLogger().setLevel(logging.INFO)
62 | run()
63 |
--------------------------------------------------------------------------------
/Python/basics/side_input_dictionary.py:
--------------------------------------------------------------------------------
1 | # Copyright 2022 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import Create, Map, pvalue
21 |
22 |
23 | def converter(elements, rates):
24 | currency, value = elements
25 | converted_value = value * rates[currency]
26 | return converted_value
27 |
28 |
29 | def run(argv=None):
30 | elements = [
31 | ("USD", 3.1415),
32 | ("USD", 1729.0),
33 | ("CHF", 2.7182),
34 | ("EUR", 1.618),
35 | ("CHF", 1.1),
36 | ("CHF", 342.45),
37 | ("EUR", 890.01),
38 | ]
39 | side_elements = [("USD", 1.0), ("EUR", 0.8), ("CHF", 0.9)]
40 |
41 | with beam.Pipeline() as p:
42 | side_input = p | "side input" >> Create(side_elements)
43 | output = (
44 | p
45 | | Create(elements)
46 | | "Convert" >> Map(converter, rates=pvalue.AsDict(side_input))
47 | | "Log" >> Map(logging.info)
48 | )
49 |
50 |
51 | if __name__ == "__main__":
52 | logging.getLogger().setLevel(logging.INFO)
53 | run()
54 |
--------------------------------------------------------------------------------
/Python/basics/side_input_iterator.py:
--------------------------------------------------------------------------------
1 | # Copyright 2022 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import Create, FlatMap, Map, pvalue
21 |
22 |
23 | def color_animal(animal, colors):
24 | for color in colors:
25 | yield f"{color} {animal}"
26 |
27 |
28 | def run(argv=None):
29 | elements = ["Dog", "Cat", "Snake", "Eagle", "Owl"]
30 | side_elements = ["Black", "While", "Brown"]
31 |
32 | with beam.Pipeline() as p:
33 | side_input = p | "side input" >> Create(side_elements)
34 | output = (
35 | p
36 | | Create(elements)
37 | | "Join colors"
38 | >> FlatMap(color_animal, colors=pvalue.AsIter(side_input))
39 | | "Log" >> Map(logging.info)
40 | )
41 |
42 |
43 | if __name__ == "__main__":
44 | logging.getLogger().setLevel(logging.INFO)
45 | run()
46 |
--------------------------------------------------------------------------------
/Python/basics/side_input_singleton.py:
--------------------------------------------------------------------------------
1 | # Copyright 2022 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import Create, Map, pvalue
21 |
22 |
23 | def run(argv=None):
24 | elements = ["Dog", "Cat", "Salmon", "Snake", "Eagle", "Owl"]
25 | side_element = ["s"]
26 |
27 | with beam.Pipeline() as p:
28 | side_input = p | "side input" >> Create(side_element)
29 | output = (
30 | p
31 | | Create(elements)
32 | | "Make plural"
33 | >> Map(
34 | lambda text, suffix: text + suffix,
35 | suffix=pvalue.AsSingleton(side_input),
36 | )
37 | | "Log" >> Map(logging.info)
38 | )
39 |
40 |
41 | if __name__ == "__main__":
42 | logging.getLogger().setLevel(logging.INFO)
43 | run()
44 |
--------------------------------------------------------------------------------
/Python/bigquery/failed_rows_bigquery.py:
--------------------------------------------------------------------------------
1 | # Copyright 2022 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import json
17 | import logging
18 |
19 | # third party libraries
20 | import apache_beam as beam
21 | from apache_beam import Filter, Map
22 | from apache_beam.io.gcp.bigquery import (
23 | BigQueryDisposition,
24 | BigQueryWriteFn,
25 | RetryStrategy,
26 | WriteToBigQuery,
27 | )
28 | from apache_beam.io.gcp.pubsub import ReadFromPubSub
29 | from apache_beam.options.pipeline_options import PipelineOptions
30 |
31 |
32 | def run(argv=None):
33 | """This pipeline shows how to access the rows that failed being
34 | inserted to BigQuery"""
35 | topic = "projects/pubsub-public-data/topics/taxirides-realtime"
36 |
37 | class FailedRowsOptions(PipelineOptions):
38 | @classmethod
39 | def _add_argparse_args(cls, parser):
40 | parser.add_argument("--output_table", help="BQ Table to write")
41 |
42 | # Schema is wrong so that rows fail
43 | wrong_schema = "test:STRING"
44 | options = FailedRowsOptions()
45 | with beam.Pipeline(options=options) as p:
46 | bq_insert = (
47 | p
48 | | "ReadFromPubSub" >> ReadFromPubSub(topic=topic)
49 | | "Load Json" >> Map(json.loads)
50 | | Filter(lambda r: r["ride_status"] == "dropoff")
51 | # Since the schema is wrong, the rows will fail
52 | | WriteToBigQuery(
53 | options.output_table,
54 | schema=wrong_schema,
55 | insert_retry_strategy=RetryStrategy.RETRY_NEVER,
56 | create_disposition=BigQueryDisposition.CREATE_IF_NEEDED,
57 | )
58 | )
59 |
60 | (
61 | bq_insert[BigQueryWriteFn.FAILED_ROWS]
62 | | "Failed" >> Map(lambda x: logging.info("Failed row: %s", x))
63 | )
64 |
65 |
66 | if __name__ == "__main__":
67 | logging.getLogger().setLevel(logging.INFO)
68 | run()
69 |
--------------------------------------------------------------------------------
/Python/bigquery/read_all_bigquery.py:
--------------------------------------------------------------------------------
1 | # Copyright 2022 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import Create, Map
21 | from apache_beam.io import ReadAllFromBigQuery, ReadFromBigQueryRequest
22 | from apache_beam.options.pipeline_options import PipelineOptions
23 |
24 |
25 | def run(argv=None):
26 | """Use ReadAllFromBigQuery to define table and query reads from
27 | BigQuery at pipeline runtime.
28 | """
29 | query1 = (
30 | "SELECT repository_language, COUNT(repository_language) totalRepos"
31 | " FROM `bigquery-public-data.samples.github_timeline` GROUP BY 1"
32 | )
33 | query2 = (
34 | "SELECT year, mother_residence_state, COUNT(*) countByYearState"
35 | " FROM `bigquery-public-data.samples.natality` GROUP BY 1, 2"
36 | )
37 | table = "bigquery-public-data:samples.shakespeare"
38 |
39 | options = PipelineOptions()
40 | with beam.Pipeline(options=options) as p:
41 | read_requests = p | Create(
42 | [
43 | ReadFromBigQueryRequest(query=query1),
44 | ReadFromBigQueryRequest(query=query2),
45 | ReadFromBigQueryRequest(table=table),
46 | ]
47 | )
48 | output = (
49 | read_requests
50 | | "ReadFromAll" >> ReadAllFromBigQuery()
51 | | "LogMessages" >> Map(logging.info)
52 | )
53 |
54 |
55 | if __name__ == "__main__":
56 | logging.getLogger().setLevel(logging.INFO)
57 | run()
58 |
--------------------------------------------------------------------------------
/Python/bigquery/read_query_bigquery.py:
--------------------------------------------------------------------------------
1 | # Copyright 2022 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import Map
21 | from apache_beam.io import ReadFromBigQuery
22 | from apache_beam.options.pipeline_options import PipelineOptions
23 |
24 |
25 | class ReadQueryOptions(PipelineOptions):
26 | @classmethod
27 | def _add_argparse_args(cls, parser):
28 | # Add a command line flag to be parsed along with other
29 | # normal PipelineOptions. This flag will store the SQL query
30 | # to be run against BigQuery.
31 | parser.add_argument(
32 | "--query",
33 | default=(
34 | "SELECT repository_language, COUNT(repository_language) totalRepos" # noqa:E501
35 | " FROM `bigquery-public-data.samples.github_timeline` GROUP BY 1" # noqa:E501
36 | ),
37 | help="BigQuery query to read data",
38 | )
39 |
40 |
41 | def run(argv=None):
42 | options = ReadQueryOptions()
43 | # Create a Beam pipeline with 2 steps:
44 | # run a query against BigQuery and log the results
45 | with beam.Pipeline(options=options) as p:
46 | output = (
47 | p
48 | | "ReadFromQuery"
49 | >> ReadFromBigQuery(query=options.query, use_standard_sql=True)
50 | | "LogData" >> Map(logging.info)
51 | )
52 |
53 |
54 | if __name__ == "__main__":
55 | logging.getLogger().setLevel(logging.INFO)
56 | run()
57 |
--------------------------------------------------------------------------------
/Python/bigquery/read_table_bigquery.py:
--------------------------------------------------------------------------------
1 | # Copyright 2022 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import Map
21 | from apache_beam.io import ReadFromBigQuery
22 | from apache_beam.options.pipeline_options import PipelineOptions
23 |
24 |
25 | class BigQueryTableOptions(PipelineOptions):
26 | @classmethod
27 | def _add_argparse_args(cls, parser):
28 | parser.add_argument(
29 | "--table",
30 | default="bigquery-public-data:samples.github_timeline",
31 | help="BigQuery table to read data from",
32 | )
33 |
34 |
35 | def run(argv=None):
36 | options = BigQueryTableOptions()
37 | with beam.Pipeline(options=options) as p:
38 | output = (
39 | p
40 | | "ReadTable" >> ReadFromBigQuery(table=options.table)
41 | | "LogData" >> Map(logging.info)
42 | )
43 |
44 |
45 | if __name__ == "__main__":
46 | logging.getLogger().setLevel(logging.INFO)
47 | run()
48 |
--------------------------------------------------------------------------------
/Python/bigquery/read_table_ref_bigquery.py:
--------------------------------------------------------------------------------
1 | # Copyright 2022 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import Map
21 | from apache_beam.io import ReadFromBigQuery
22 | from apache_beam.io.gcp.internal.clients import bigquery
23 |
24 |
25 | def run(argv=None):
26 | # Configure the table we are reading from.
27 | table = bigquery.TableReference(
28 | projectId="bigquery-public-data",
29 | datasetId="samples",
30 | tableId="github_timeline",
31 | )
32 |
33 | # Create a Beam pipeline with 2 steps: read from BigQuery and log the data.
34 | with beam.Pipeline() as p:
35 | output = (
36 | p
37 | | "ReadTable" >> ReadFromBigQuery(table=table)
38 | | "LogData" >> Map(lambda data: logging.info(data))
39 | )
40 |
41 |
42 | if __name__ == "__main__":
43 | logging.getLogger().setLevel(logging.INFO)
44 | run()
45 |
--------------------------------------------------------------------------------
/Python/bigquery/streaming_inserts_bigquery.py:
--------------------------------------------------------------------------------
1 | # Copyright 2022 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import json
17 | import logging
18 |
19 | # third party libraries
20 | import apache_beam as beam
21 | from apache_beam import Map
22 | from apache_beam.io.gcp.bigquery import BigQueryDisposition, WriteToBigQuery
23 | from apache_beam.io.gcp.pubsub import ReadFromPubSub
24 | from apache_beam.options.pipeline_options import PipelineOptions
25 |
26 |
27 | def make_row(element):
28 | return {
29 | "ride_status": element["ride_status"],
30 | "passenger_count": element["passenger_count"],
31 | "meter_reading": element["meter_reading"],
32 | "timestamp": element["timestamp"],
33 | }
34 |
35 |
36 | def run(argv=None):
37 | topic = "projects/pubsub-public-data/topics/taxirides-realtime"
38 |
39 | class StreamingInsertsOptions(PipelineOptions):
40 | @classmethod
41 | def _add_argparse_args(cls, parser):
42 | parser.add_argument("--output_table", help="BQ Table to write")
43 |
44 | table_schema = "ride_status:STRING, passenger_count:INTEGER, meter_reading:FLOAT, timestamp:STRING" # noqa:E501
45 | options = StreamingInsertsOptions()
46 | with beam.Pipeline(options=options) as p:
47 | output = (
48 | p
49 | | "ReadFromPubSub" >> ReadFromPubSub(topic=topic)
50 | | "Load Json" >> Map(json.loads)
51 | | "Select Fields" >> Map(make_row)
52 | # Defaults to StreamingInserts
53 | | WriteToBigQuery(
54 | options.output_table,
55 | schema=table_schema,
56 | write_disposition=BigQueryDisposition.WRITE_APPEND,
57 | create_disposition=BigQueryDisposition.CREATE_IF_NEEDED,
58 | )
59 | )
60 |
61 |
62 | if __name__ == "__main__":
63 | logging.getLogger().setLevel(logging.INFO)
64 | run()
65 |
--------------------------------------------------------------------------------
/Python/bigquery/write_bigquery.py:
--------------------------------------------------------------------------------
1 | # Copyright 2022 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the spe
13 |
14 | # standard libraries
15 | import logging
16 |
17 | # third party libraries
18 | import apache_beam as beam
19 | from apache_beam import Create, Map
20 | from apache_beam.io.gcp.bigquery import BigQueryDisposition, WriteToBigQuery
21 | from apache_beam.options.pipeline_options import PipelineOptions
22 |
23 |
24 | def make_row(element):
25 | """Parse element to create a row using Dict"""
26 | row_fields = element.split(", ")
27 | return {
28 | "name": row_fields[0],
29 | "year": int(row_fields[1]),
30 | "country": row_fields[2],
31 | }
32 |
33 |
34 | def run(argv=None):
35 | elements = [
36 | "Charles, 1995, USA",
37 | "Alice, 1997, Spain",
38 | "Bob, 1995, USA",
39 | "Amanda, 1991, France",
40 | "Alex, 1999, Mexico",
41 | "Eliza, 2000, Japan",
42 | ]
43 |
44 | class WriteBigQueryOptions(PipelineOptions):
45 | @classmethod
46 | def _add_argparse_args(cls, parser):
47 | # Add a command line flag to be parsed along
48 | # with other normal PipelineOptions
49 | parser.add_argument(
50 | "--output_table", required=True, help="BQ Table to write"
51 | )
52 |
53 | # define the BigQuery table schema
54 | table_schema = "name:STRING, year:INTEGER, country:STRING"
55 | options = WriteBigQueryOptions()
56 |
57 | # run the pipeline to write data into a BigQuery table
58 | with beam.Pipeline(options=options) as p:
59 | output = (
60 | p
61 | | Create(elements)
62 | | Map(make_row)
63 | | WriteToBigQuery(
64 | options.output_table,
65 | schema=table_schema,
66 | write_disposition=BigQueryDisposition.WRITE_TRUNCATE,
67 | create_disposition=BigQueryDisposition.CREATE_IF_NEEDED,
68 | )
69 | )
70 |
71 |
72 | if __name__ == "__main__":
73 | logging.getLogger().setLevel(logging.INFO)
74 | run()
75 |
--------------------------------------------------------------------------------
/Python/bigquery/write_partitioned_cluster_table_bigquery.py:
--------------------------------------------------------------------------------
1 | # Copyright 2022 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import Create, Map
21 | from apache_beam.io.gcp.bigquery import BigQueryDisposition, WriteToBigQuery
22 | from apache_beam.options.pipeline_options import PipelineOptions
23 |
24 |
25 | def make_row(element):
26 | row_fields = element.split(", ")
27 | return {
28 | "name": row_fields[0],
29 | "date": row_fields[1],
30 | "country": row_fields[2],
31 | }
32 |
33 |
34 | def run(argv=None):
35 | elements = [
36 | "Charles, 2021-02-01, USA",
37 | "Alice, 2021-02-04, Spain",
38 | "Bob, 2021-02-02, USA",
39 | "Amanda, 2021-02-07, France",
40 | "Alex, 2021-02-06, Mexico",
41 | "Eliza, 2021-02-02, Japan",
42 | ]
43 |
44 | class WriteBigQueryOptions(PipelineOptions):
45 | @classmethod
46 | def _add_argparse_args(cls, parser):
47 | parser.add_argument("--output_table", help="BQ Table to write")
48 |
49 | table_schema = "name:STRING, date:DATE, country:STRING"
50 | options = WriteBigQueryOptions()
51 | with beam.Pipeline(options=options) as p:
52 | output = (
53 | p
54 | | Create(elements)
55 | | Map(make_row)
56 | | WriteToBigQuery(
57 | options.output_table,
58 | schema=table_schema,
59 | write_disposition=BigQueryDisposition.WRITE_TRUNCATE,
60 | create_disposition=BigQueryDisposition.CREATE_IF_NEEDED,
61 | additional_bq_parameters={
62 | "timePartitioning": {"type": "DAY"},
63 | "clustering": {"fields": ["country"]},
64 | },
65 | )
66 | )
67 |
68 |
69 | if __name__ == "__main__":
70 | logging.getLogger().setLevel(logging.INFO)
71 | run()
72 |
--------------------------------------------------------------------------------
/Python/bigtable/read_bigtable.py:
--------------------------------------------------------------------------------
1 | # Copyright 2023 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 | import apache_beam as beam
18 |
19 | # third party libraries
20 | from apache_beam import Map
21 | from apache_beam.io.gcp.bigtableio import ReadFromBigtable
22 | from apache_beam.options.pipeline_options import PipelineOptions
23 |
24 |
25 | class BigtableOptions(PipelineOptions):
26 | @classmethod
27 | def _add_argparse_args(cls, parser):
28 | # Add a command line flag to be parsed along
29 | # with other normal PipelineOptions
30 | parser.add_argument(
31 | "--project_id",
32 | required=True,
33 | help="Project ID"
34 | )
35 | parser.add_argument(
36 | "--instance_id",
37 | required=True,
38 | help="Cloud Bigtable instance ID"
39 | )
40 | parser.add_argument(
41 | "--table_id",
42 | required=True,
43 | help="Cloud Bigtable table ID"
44 | )
45 |
46 |
47 | def run():
48 | """
49 | This pipeline shows how to read from Cloud Bigtable.
50 | """
51 |
52 | options = BigtableOptions()
53 |
54 | with beam.Pipeline(options=options) as p:
55 |
56 | output = (
57 | p
58 | | "Read from Bigtable" >> ReadFromBigtable(
59 | project_id=options.project_id,
60 | instance_id=options.instance_id,
61 | table_id=options.table_id
62 | )
63 | | "Extract cells" >> beam.Map(
64 | lambda row: row._cells
65 | )
66 | | "Log Data" >> Map(logging.info)
67 | )
68 |
69 |
70 | if __name__ == "__main__":
71 | logging.getLogger().setLevel(logging.INFO)
72 | run()
73 |
--------------------------------------------------------------------------------
/Python/csv/read_csv.py:
--------------------------------------------------------------------------------
1 | # Copyright 2023 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import Map
21 | from apache_beam.io.textio import ReadFromCsv
22 | from apache_beam.options.pipeline_options import PipelineOptions
23 |
24 |
25 | class CsvOptions(PipelineOptions):
26 | @classmethod
27 | def _add_argparse_args(cls, parser):
28 | # Add a command line flag to be parsed along
29 | # with other normal PipelineOptions
30 | parser.add_argument(
31 | "--file_path",
32 | required=True,
33 | help="Csv file path"
34 | )
35 |
36 |
37 | def run():
38 | """
39 | This pipeline shows how to read from Csv file.
40 | """
41 |
42 | options = CsvOptions()
43 |
44 | with beam.Pipeline(options=options) as p:
45 |
46 | output = (
47 | p
48 | | "Read from Csv file" >> ReadFromCsv(
49 | path=options.file_path
50 | )
51 | | "Log Data" >> Map(logging.info)
52 | )
53 |
54 |
55 | if __name__ == "__main__":
56 | logging.getLogger().setLevel(logging.INFO)
57 | run()
58 |
--------------------------------------------------------------------------------
/Python/csv/write_csv.py:
--------------------------------------------------------------------------------
1 | # Copyright 2023 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import Create, Map
21 | from apache_beam.io.textio import WriteToCsv
22 | from apache_beam.options.pipeline_options import PipelineOptions
23 |
24 |
25 | class CsvOptions(PipelineOptions):
26 | @classmethod
27 | def _add_argparse_args(cls, parser):
28 | # Add a command line flag to be parsed along
29 | # with other normal PipelineOptions
30 | parser.add_argument(
31 | "--file_path",
32 | required=True,
33 | help="Csv file path"
34 | )
35 |
36 |
37 | def run():
38 | """
39 | This pipeline shows how to write to Csv file.
40 | """
41 |
42 | options = CsvOptions()
43 |
44 | with beam.Pipeline(options=options) as p:
45 |
46 | elements = [
47 | (1, "Charles"),
48 | (2, "Alice"),
49 | (3, "Bob"),
50 | (4, "Amanda"),
51 | (5, "Alex"),
52 | (6, "Eliza")
53 | ]
54 |
55 | output = (
56 | p
57 | | "Create" >> Create(elements)
58 | | "Map to Row" >> Map(map_to_row)
59 | | "Write to Csv file" >> WriteToCsv(
60 | path=options.file_path
61 | )
62 | )
63 |
64 |
65 | def map_to_row(element):
66 | """
67 | Converts a given element into a Beam Row object.
68 | """
69 |
70 | return beam.Row(id=element[0], name=element[1])
71 |
72 |
73 | if __name__ == "__main__":
74 | logging.getLogger().setLevel(logging.INFO)
75 | run()
76 |
--------------------------------------------------------------------------------
/Python/extra_examples/file_system_dynamics.py:
--------------------------------------------------------------------------------
1 | # Copyright 2022 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import Create, DoFn, ParDo
21 | from apache_beam.io.filesystems import FileSystems
22 | from apache_beam.options.pipeline_options import PipelineOptions
23 |
24 |
25 | class WriteFileSystems(DoFn):
26 | def process(self, element):
27 | # Beam's built-in FileSystems module has built in support for many
28 | # different backing storage systems, we use this to write our element.
29 | # Each input element is formatted as a Tuple of the form
30 | #
31 | writer = FileSystems.create(element[0])
32 | writer.write(bytes(element[1], encoding="utf8"))
33 | writer.close()
34 |
35 |
36 | def run(argv=None):
37 | # TODO: add your bucket names
38 | bucket1 = ""
39 | bucket2 = ""
40 | elements = [
41 | ("gs://" + bucket1 + "/beam/dynamic.txt", "line"),
42 | ("gs://" + bucket2 + "/beam/dynamic.txt", "line"),
43 | ]
44 | options = PipelineOptions(save_main_session=True)
45 | with beam.Pipeline(options=options) as p:
46 | output = (
47 | p
48 | | Create(elements)
49 | | "Write FileSystems" >> ParDo(WriteFileSystems())
50 | )
51 |
52 |
53 | if __name__ == "__main__":
54 | logging.getLogger().setLevel(logging.INFO)
55 | run()
56 |
--------------------------------------------------------------------------------
/Python/file/match_files.py:
--------------------------------------------------------------------------------
1 | # Copyright 2023 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import Map
21 | from apache_beam.io.fileio import MatchFiles
22 | from apache_beam.options.pipeline_options import PipelineOptions
23 |
24 |
25 | class FilesOptions(PipelineOptions):
26 | @classmethod
27 | def _add_argparse_args(cls, parser):
28 | # Add a command line flag to be parsed along
29 | # with other normal PipelineOptions
30 | parser.add_argument(
31 | "--file_pattern",
32 | required=True,
33 | help="File pattern"
34 | )
35 |
36 |
37 | def run():
38 | """
39 | This pipeline shows how to read matching files
40 | in the form of ``FileMetadata`` objects.
41 | """
42 |
43 | options = FilesOptions()
44 |
45 | with beam.Pipeline(options=options) as p:
46 |
47 | output = (
48 | p
49 | | "Match files" >> MatchFiles(
50 | file_pattern=options.file_pattern
51 | )
52 | | "Log Data" >> Map(logging.info)
53 | )
54 |
55 |
56 | if __name__ == "__main__":
57 | logging.getLogger().setLevel(logging.INFO)
58 | run()
59 |
--------------------------------------------------------------------------------
/Python/file/match_files_continuously.py:
--------------------------------------------------------------------------------
1 | # Copyright 2023 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import Map
21 | from apache_beam.io.fileio import MatchContinuously
22 | from apache_beam.options.pipeline_options import PipelineOptions
23 |
24 |
25 | class FilesOptions(PipelineOptions):
26 | @classmethod
27 | def _add_argparse_args(cls, parser):
28 | # Add a command line flag to be parsed along
29 | # with other normal PipelineOptions
30 | parser.add_argument(
31 | "--file_pattern",
32 | required=True,
33 | help="File pattern"
34 | )
35 |
36 |
37 | def run():
38 | """
39 | This pipeline shows how to read matching files
40 | in the form of ``FileMetadata`` objects continuously.
41 | """
42 |
43 | options = FilesOptions()
44 |
45 | with beam.Pipeline(options=options) as p:
46 |
47 | output = (
48 | p
49 | | "Match files" >> MatchContinuously(
50 | file_pattern=options.file_pattern,
51 | interval=10
52 | )
53 | | "Log Data" >> Map(logging.info)
54 | )
55 |
56 |
57 | if __name__ == "__main__":
58 | logging.getLogger().setLevel(logging.INFO)
59 | run()
60 |
--------------------------------------------------------------------------------
/Python/file/write_files.py:
--------------------------------------------------------------------------------
1 | # Copyright 2023 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import Create, Map
21 | from apache_beam.io.fileio import WriteToFiles
22 | from apache_beam.options.pipeline_options import PipelineOptions
23 |
24 |
25 | class FilesOptions(PipelineOptions):
26 | @classmethod
27 | def _add_argparse_args(cls, parser):
28 | # Add a command line flag to be parsed along
29 | # with other normal PipelineOptions
30 | parser.add_argument(
31 | "--path",
32 | required=True,
33 | help="The directory to write files into."
34 | )
35 |
36 |
37 | def run():
38 | """
39 | This pipeline shows how to write to a set of output files.
40 | """
41 |
42 | options = FilesOptions()
43 |
44 | with beam.Pipeline(options=options) as p:
45 | elements = [
46 | {'id': 1, 'name': 'Charles'},
47 | {'id': 2, 'name': 'Alice'},
48 | {'id': 3, 'name': 'Bob'},
49 | {'id': 4, 'name': 'Amanda'},
50 | {'id': 5, 'name': 'Alex'},
51 | {'id': 6, 'name': 'Eliza'}
52 | ]
53 |
54 | output = (
55 | p
56 | | "Create" >> Create(elements)
57 | | "Serialize" >> Map(map_to_json)
58 | | "Write to files" >> WriteToFiles(
59 | path=options.path
60 | )
61 | )
62 |
63 |
64 | def map_to_json(element):
65 | """
66 | Converts a given input element into a JSON string.
67 | """
68 |
69 | # third party libraries
70 | import json
71 |
72 | return json.dumps(element)
73 |
74 |
75 | if __name__ == "__main__":
76 | logging.getLogger().setLevel(logging.INFO)
77 | run()
78 |
--------------------------------------------------------------------------------
/Python/gcs/match_continuously.py:
--------------------------------------------------------------------------------
1 | # Copyright 2022 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import Map
21 | from apache_beam.io.fileio import MatchContinuously
22 | from apache_beam.io.textio import ReadAllFromText
23 | from apache_beam.options.pipeline_options import PipelineOptions
24 |
25 |
26 | def run(argv=None):
27 | class MatchContinuouslyOptions(PipelineOptions):
28 | @classmethod
29 | def _add_argparse_args(cls, parser):
30 | parser.add_argument(
31 | "--pattern", help="Pattern to check for new files"
32 | )
33 |
34 | parser.add_argument(
35 | "--interval", default=300, help="Frequency to look for files"
36 | )
37 |
38 | options = MatchContinuouslyOptions(streaming=True)
39 |
40 | with beam.Pipeline(options=options) as p:
41 | (
42 | p
43 | | MatchContinuously(options.pattern, interval=options.interval)
44 | | Map(lambda x: x.path)
45 | | ReadAllFromText()
46 | | Map(logging.info)
47 | )
48 |
49 |
50 | if __name__ == "__main__":
51 | logging.getLogger().setLevel(logging.INFO)
52 | run()
53 |
--------------------------------------------------------------------------------
/Python/gcs/read_all_textio.py:
--------------------------------------------------------------------------------
1 | # Copyright 2022 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import Create, Map
21 | from apache_beam.io.textio import ReadAllFromText
22 | from apache_beam.options.pipeline_options import PipelineOptions
23 | from apache_beam.transforms.combiners import Count
24 |
25 |
26 | def run(argv=None):
27 | # Define the text file GCS locations.
28 | # You should replace these locations with the paths to your text files.
29 | elements = [
30 | "gs://apache-beam-samples/shakespeare/kinglear.txt",
31 | "gs://apache-beam-samples/shakespeare/macbeth.txt",
32 | "gs://apache-beam-samples/shakespeare/a*",
33 | ]
34 |
35 | options = PipelineOptions()
36 |
37 | # ReadAllFromText reads the files from elements
38 | # and parse each file as newline-delimited elements
39 | # At last, Count. globally() counts the total number of lines for all files
40 | with beam.Pipeline(options=options) as p:
41 | (
42 | p
43 | | Create(elements)
44 | | "Read File from GCS" >> ReadAllFromText()
45 | | Count.Globally()
46 | | "Log" >> Map(lambda x: logging.info("Total lines %d", x))
47 | )
48 |
49 |
50 | if __name__ == "__main__":
51 | logging.getLogger().setLevel(logging.INFO)
52 | run()
53 |
--------------------------------------------------------------------------------
/Python/gcs/read_all_textio_streaming.py:
--------------------------------------------------------------------------------
1 | # Copyright 2022 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import Map, WindowInto
21 | from apache_beam.io import ReadFromPubSub
22 | from apache_beam.io.textio import ReadAllFromText
23 | from apache_beam.options.pipeline_options import PipelineOptions
24 | from apache_beam.transforms import window
25 | from apache_beam.transforms.combiners import Count
26 |
27 |
28 | def run(argv=None):
29 | class ReadAllStreamingTextOptions(PipelineOptions):
30 | @classmethod
31 | def _add_argparse_args(cls, parser):
32 | parser.add_argument(
33 | "--topic",
34 | description="Provide a topic to read from",
35 | dest="topic",
36 | help="You need to create this topic and the messages will be GCS paths.", # noqa:E501
37 | )
38 |
39 | options = ReadAllStreamingTextOptions(streaming=True)
40 |
41 | with beam.Pipeline(options=options) as p:
42 | (
43 | p
44 | | "Read GCS paths" >> ReadFromPubSub(topic=options.topic)
45 | | "Decode messages" >> Map(lambda x: x.decode("utf-8"))
46 | | "Read File from GCS" >> ReadAllFromText()
47 | | WindowInto(window.FixedWindows(300)) # 5 * 60sec = 5min
48 | | "Count Files" >> Count.Globally().without_defaults()
49 | | "Log" >> Map(lambda x: logging.info("Total lines %d", x))
50 | )
51 |
52 |
53 | if __name__ == "__main__":
54 | logging.getLogger().setLevel(logging.INFO)
55 | run()
56 |
--------------------------------------------------------------------------------
/Python/gcs/read_avro.py:
--------------------------------------------------------------------------------
1 | # Copyright 2022 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import Map
21 | from apache_beam.io.avroio import ReadFromAvro
22 | from apache_beam.options.pipeline_options import PipelineOptions
23 |
24 |
25 | def run(argv=None):
26 | class ReadAvroOptions(PipelineOptions):
27 | @classmethod
28 | def _add_argparse_args(cls, parser):
29 | parser.add_argument(
30 | "--path",
31 | default="gs://cloud-samples-data/bigquery/us-states/*.avro",
32 | help="GCS path to read from",
33 | )
34 |
35 | options = ReadAvroOptions()
36 |
37 | with beam.Pipeline(options=options) as p:
38 | (
39 | p
40 | | "ReadAvro" >> ReadFromAvro(options.path)
41 | | "CheckRow"
42 | >> Map(
43 | lambda row: f"The abbreviation of {row['name']} is {row['post_abbr']}" # noqa:E501
44 | )
45 | | Map(logging.info)
46 | )
47 |
48 |
49 | if __name__ == "__main__":
50 | logging.getLogger().setLevel(logging.INFO)
51 | run()
52 |
--------------------------------------------------------------------------------
/Python/gcs/read_parquet.py:
--------------------------------------------------------------------------------
1 | # Copyright 2022 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import Map
21 | from apache_beam.io.parquetio import ReadFromParquet
22 | from apache_beam.options.pipeline_options import PipelineOptions
23 |
24 |
25 | def run(argv=None):
26 | class ReadParquetOptions(PipelineOptions):
27 | @classmethod
28 | def _add_argparse_args(cls, parser):
29 | parser.add_argument(
30 | "--path",
31 | default="gs://cloud-samples-data/bigquery/us-states/*.parquet",
32 | help="GCS path to read from",
33 | )
34 |
35 | options = ReadParquetOptions()
36 |
37 | with beam.Pipeline(options=options) as p:
38 | (
39 | p
40 | | "ReadParquet" >> ReadFromParquet(options.path)
41 | | "CheckRow"
42 | >> Map(
43 | lambda row: f"The abbreviation of {row['name']} is {row['post_abbr']}" # noqa:E501
44 | )
45 | | Map(logging.info)
46 | )
47 |
48 |
49 | if __name__ == "__main__":
50 | logging.getLogger().setLevel(logging.INFO)
51 | run()
52 |
--------------------------------------------------------------------------------
/Python/gcs/read_textio.py:
--------------------------------------------------------------------------------
1 | # Copyright 2022 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import Map
21 | from apache_beam.io.textio import ReadFromText
22 | from apache_beam.options.pipeline_options import PipelineOptions
23 | from apache_beam.transforms.combiners import Count
24 |
25 |
26 | def run(argv=None):
27 | class ReadTextOptions(PipelineOptions):
28 | @classmethod
29 | def _add_argparse_args(cls, parser):
30 | # Add a command line flag to be parsed along
31 | # with other normal PipelineOptions
32 | parser.add_argument(
33 | "--path",
34 | default="gs://dataflow-samples/shakespeare/kinglear.txt",
35 | help="GCS path to read",
36 | )
37 |
38 | options = ReadTextOptions()
39 |
40 | # Create a Beam pipeline with 3 steps:
41 | # 1) Read text. This will emit one record per line
42 | # 2) Count.Globally(). This will count the number of
43 | # elements in the PCollection.
44 | # 3) Log the output.
45 | with beam.Pipeline(options=options) as p:
46 | (
47 | p
48 | | "ReadText" >> ReadFromText(options.path)
49 | | Count.Globally()
50 | | "Log" >> Map(lambda x: logging.info("Total lines %d", x))
51 | )
52 |
53 |
54 | if __name__ == "__main__":
55 | logging.getLogger().setLevel(logging.INFO)
56 | run()
57 |
--------------------------------------------------------------------------------
/Python/gcs/write_avro.py:
--------------------------------------------------------------------------------
1 | # Copyright 2022 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import Create
21 | from apache_beam.io.avroio import WriteToAvro
22 | from apache_beam.options.pipeline_options import PipelineOptions
23 |
24 |
25 | def run(argv=None):
26 | class WriteAvroOptions(PipelineOptions):
27 | @classmethod
28 | def _add_argparse_args(cls, parser):
29 | parser.add_argument("--output", help="GCS path to write")
30 |
31 | options = WriteAvroOptions()
32 |
33 | schema = {
34 | "namespace": "example.avro",
35 | "type": "record",
36 | "name": "People",
37 | "fields": [
38 | {"name": "name", "type": "string"},
39 | {"name": "age", "type": "long"},
40 | {"name": "job", "type": "string"},
41 | ],
42 | }
43 |
44 | elements = [
45 | {"name": "Maria", "age": 19, "job": "CEO"},
46 | {"name": "Sara", "age": 44, "job": "Medic"},
47 | {"name": "Juan", "age": 31, "job": "Data Engineer"},
48 | {"name": "Kim", "age": 25, "job": "Lawyer"},
49 | {"name": "Roger", "age": 99, "job": "Pipeline Fixer"},
50 | ]
51 |
52 | with beam.Pipeline(options=options) as p:
53 | (
54 | p
55 | | Create(elements)
56 | # Input has to be a dict
57 | | WriteToAvro(
58 | options.output,
59 | schema=schema,
60 | )
61 | )
62 |
63 |
64 | if __name__ == "__main__":
65 | logging.getLogger().setLevel(logging.INFO)
66 | run()
67 |
--------------------------------------------------------------------------------
/Python/gcs/write_parquet.py:
--------------------------------------------------------------------------------
1 | # Copyright 2022 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | import pyarrow
21 | from apache_beam import Create
22 | from apache_beam.io.parquetio import WriteToParquet
23 | from apache_beam.options.pipeline_options import PipelineOptions
24 |
25 |
26 | def run(argv=None):
27 | class WriteParquetOptions(PipelineOptions):
28 | @classmethod
29 | def _add_argparse_args(cls, parser):
30 | parser.add_argument("--output", help="GCS path to write")
31 |
32 | options = WriteParquetOptions()
33 |
34 | schema = pyarrow.schema(
35 | [
36 | # name, type, nullable=True
37 | ("name", pyarrow.string(), False),
38 | ("age", pyarrow.int64(), False),
39 | ("job", pyarrow.string()),
40 | ]
41 | )
42 |
43 | elements = [
44 | {"name": "Maria", "age": 19, "job": "CEO"},
45 | {"name": "Sara", "age": 44, "job": "Medic"},
46 | {"name": "Juan", "age": 31, "job": "Data Engineer"},
47 | {"name": "Kim", "age": 25, "job": "Lawyer"},
48 | {"name": "Roger", "age": 99, "job": "Pipeline Fixer"},
49 | ]
50 |
51 | with beam.Pipeline(options=options) as p:
52 | (
53 | p
54 | | Create(elements)
55 | # Input has to be a dict
56 | | WriteToParquet(options.output, schema=schema, num_shards=2)
57 | )
58 |
59 |
60 | if __name__ == "__main__":
61 | logging.getLogger().setLevel(logging.INFO)
62 | run()
63 |
--------------------------------------------------------------------------------
/Python/gcs/write_textio.py:
--------------------------------------------------------------------------------
1 | # Copyright 2022 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import Create, Map
21 | from apache_beam.io.textio import WriteToText
22 | from apache_beam.options.pipeline_options import PipelineOptions
23 |
24 |
25 | def run(argv=None):
26 | class WriteTextOptions(PipelineOptions):
27 | @classmethod
28 | def _add_argparse_args(cls, parser):
29 | parser.add_argument(
30 | "--output",
31 | description="Output to write to",
32 | dest="output",
33 | required=True,
34 | help="Output file to write results to.",
35 | )
36 | parser.add_argument(
37 | "--max_number",
38 | dest="max_number",
39 | default=100,
40 | help="Number of shards to process.",
41 | )
42 |
43 | options = WriteTextOptions()
44 | elements = range(options.max_number)
45 |
46 | with beam.Pipeline(options=options) as p:
47 | (
48 | p
49 | | Create(elements)
50 | | "Format String"
51 | >> Map(
52 | lambda x: f"This element number is {x}"
53 | ) # Changing number to string and
54 | | "Write Files" >> WriteToText(options.output)
55 | )
56 |
57 |
58 | if __name__ == "__main__":
59 | logging.getLogger().setLevel(logging.INFO)
60 | run()
61 |
--------------------------------------------------------------------------------
/Python/json/read_json.py:
--------------------------------------------------------------------------------
1 | # Copyright 2023 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import Map
21 | from apache_beam.io.textio import ReadFromJson
22 | from apache_beam.options.pipeline_options import PipelineOptions
23 |
24 |
25 | class JsonOptions(PipelineOptions):
26 | @classmethod
27 | def _add_argparse_args(cls, parser):
28 | # Add a command line flag to be parsed along
29 | # with other normal PipelineOptions
30 | parser.add_argument(
31 | "--file_path",
32 | default="gs://your-bucket/your-file.json",
33 | help="Json file path"
34 | )
35 |
36 |
37 | def run():
38 | """
39 | This pipeline shows how to read from Json file.
40 | """
41 |
42 | options = JsonOptions()
43 |
44 | with beam.Pipeline(options=options) as p:
45 |
46 | output = (
47 | p
48 | | "Read from Json file" >> ReadFromJson(
49 | path=options.file_path,
50 | lines=False
51 | )
52 | | "Log Data" >> Map(logging.info)
53 | )
54 |
55 |
56 | if __name__ == "__main__":
57 | logging.getLogger().setLevel(logging.INFO)
58 | run()
59 |
--------------------------------------------------------------------------------
/Python/json/write_json.py:
--------------------------------------------------------------------------------
1 | # Copyright 2023 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import Create, Map
21 | from apache_beam.io.textio import WriteToJson
22 | from apache_beam.options.pipeline_options import PipelineOptions
23 |
24 |
25 | class JsonOptions(PipelineOptions):
26 | @classmethod
27 | def _add_argparse_args(cls, parser):
28 | # Add a command line flag to be parsed along
29 | # with other normal PipelineOptions
30 | parser.add_argument(
31 | "--file_path",
32 | default="gs://your-bucket/your-file.json",
33 | help="Json file path"
34 | )
35 |
36 |
37 | def run():
38 | """
39 | This pipeline shows how to write to Json file.
40 | """
41 |
42 | options = JsonOptions()
43 |
44 | with beam.Pipeline(options=options) as p:
45 |
46 | elements = [
47 | (1, "Charles"),
48 | (2, "Alice"),
49 | (3, "Bob"),
50 | (4, "Amanda"),
51 | (5, "Alex"),
52 | (6, "Eliza")
53 | ]
54 |
55 | output = (
56 | p
57 | | "Create" >> Create(elements)
58 | | "Map to Row" >> Map(map_to_row)
59 | | "Write to Json file" >> WriteToJson(
60 | path=options.file_path
61 | )
62 | )
63 |
64 |
65 | def map_to_row(element):
66 | """
67 | Converts a given element into a Beam Row object.
68 | """
69 | return beam.Row(id=element[0], name=element[1])
70 |
71 |
72 | if __name__ == "__main__":
73 | logging.getLogger().setLevel(logging.INFO)
74 | run()
75 |
--------------------------------------------------------------------------------
/Python/kafka/read_kafka.py:
--------------------------------------------------------------------------------
1 | # Copyright 2023 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import Map
21 | from apache_beam.io.kafka import ReadFromKafka
22 | from apache_beam.options.pipeline_options import PipelineOptions
23 |
24 |
25 | class KafkaOptions(PipelineOptions):
26 | @classmethod
27 | def _add_argparse_args(cls, parser):
28 | # Add a command line flag to be parsed along
29 | # with other normal PipelineOptions
30 | parser.add_argument(
31 | "--bootstrap_servers",
32 | default="localhost:9092",
33 | help="Apache Kafka bootstrap servers"
34 | )
35 | parser.add_argument(
36 | "--topic",
37 | default="your-topic",
38 | help="Apache Kafka topic"
39 | )
40 |
41 |
42 | def run():
43 | """
44 | This pipeline shows how to read from Apache Kafka.
45 | """
46 |
47 | options = KafkaOptions()
48 |
49 | with beam.Pipeline(options=options) as p:
50 |
51 | output = (
52 | p
53 | | "Read from Kafka" >> ReadFromKafka(
54 | consumer_config={
55 | "bootstrap.servers": options.bootstrap_servers
56 | },
57 | topics=[options.topic],
58 | with_metadata=False
59 | )
60 | | "Log Data" >> Map(logging.info)
61 | )
62 |
63 |
64 | if __name__ == "__main__":
65 | logging.getLogger().setLevel(logging.INFO)
66 | run()
67 |
--------------------------------------------------------------------------------
/Python/minimal/create.py:
--------------------------------------------------------------------------------
1 | # Copyright 2022 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import Create
21 |
22 |
23 | def run(argv=None):
24 | # Use range(10) to generate a list of numbers up to 10.
25 | # Create a PCollection with beam.Create and finally
26 | # use the print function in beam.Map to print them.
27 | with beam.Pipeline() as p:
28 | p | Create(range(10)) | beam.Map(print)
29 |
30 |
31 | if __name__ == "__main__":
32 | logging.getLogger().setLevel(logging.INFO)
33 | run()
34 |
--------------------------------------------------------------------------------
/Python/minimal/create_parameters.py:
--------------------------------------------------------------------------------
1 | # Copyright 2022 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import Create, Map
21 | from apache_beam.options.pipeline_options import PipelineOptions
22 |
23 |
24 | def run(argv=None):
25 | class CreateOptions(PipelineOptions):
26 | @classmethod
27 | def _add_argparse_args(cls, parser):
28 | parser.add_argument(
29 | "--max_n",
30 | help="Upper bound for range in Create",
31 | default=10,
32 | type=int,
33 | )
34 |
35 | options = CreateOptions()
36 |
37 | with beam.Pipeline(options=options) as p:
38 | (p | Create(range(options.max_n)) | "Map" >> Map(lambda x: x))
39 |
40 |
41 | if __name__ == "__main__":
42 | logging.getLogger().setLevel(logging.INFO)
43 | run()
44 |
--------------------------------------------------------------------------------
/Python/minimal/streaming.py:
--------------------------------------------------------------------------------
1 | # Copyright 2022 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import Map
21 | from apache_beam.io.gcp.pubsub import ReadFromPubSub
22 | from apache_beam.options.pipeline_options import PipelineOptions
23 |
24 |
25 | def run(argv=None):
26 | options = PipelineOptions(streaming=True)
27 |
28 | with beam.Pipeline(options=options) as p:
29 | topic = "projects/pubsub-public-data/topics/taxirides-realtime"
30 |
31 | (p | ReadFromPubSub(topic=topic) | "Map" >> Map(lambda x: x))
32 |
33 | p.run()
34 |
35 |
36 | if __name__ == "__main__":
37 | logging.getLogger().setLevel(logging.INFO)
38 | run()
39 |
--------------------------------------------------------------------------------
/Python/mongodb/read_mongodb.py:
--------------------------------------------------------------------------------
1 | # Copyright 2023 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import Map
21 | from apache_beam.io.mongodbio import ReadFromMongoDB
22 | from apache_beam.options.pipeline_options import PipelineOptions
23 |
24 |
25 | class MongoDBOptions(PipelineOptions):
26 | @classmethod
27 | def _add_argparse_args(cls, parser):
28 | # Add a command line flag to be parsed along
29 | # with other normal PipelineOptions
30 | parser.add_argument(
31 | "--uri",
32 | default="mongodb://localhost:27017",
33 | help="The MongoDB connection string following the URI format"
34 | )
35 | parser.add_argument(
36 | "--db_name",
37 | required=True,
38 | help="The MongoDB database name"
39 | )
40 | parser.add_argument(
41 | "--collection",
42 | required=True,
43 | help="The MongoDB collection name"
44 | )
45 |
46 |
47 | def run():
48 | """
49 | This pipeline shows how to read from MongoDB.
50 | """
51 |
52 | options = MongoDBOptions()
53 |
54 | with beam.Pipeline(options=options) as p:
55 |
56 | output = (
57 | p
58 | | "Read from MongoDB" >> ReadFromMongoDB(
59 | uri=options.uri,
60 | db=options.db_name,
61 | coll=options.collection
62 | )
63 | | "Log Data" >> Map(logging.info)
64 | )
65 |
66 |
67 | if __name__ == "__main__":
68 | logging.getLogger().setLevel(logging.INFO)
69 | run()
70 |
--------------------------------------------------------------------------------
/Python/pubsub/read_pubsub_multiple.py:
--------------------------------------------------------------------------------
1 | # Copyright 2022 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import Map
21 | from apache_beam.io import MultipleReadFromPubSub, PubSubSourceDescriptor
22 | from apache_beam.options.pipeline_options import PipelineOptions
23 |
24 |
25 | def run():
26 | class ReadPubSubOptions(PipelineOptions):
27 | @classmethod
28 | def _add_argparse_args(cls, parser):
29 | # Add a command line flag to be parsed along
30 | # with other normal PipelineOptions
31 | parser.add_argument(
32 | "--sources",
33 | required=True,
34 | help="PubSub topics or subscriptions, separated by a comma,"
35 | "e.g.: projects/a/topics/t1,projects/a/topics/t2.",
36 | )
37 |
38 | options = ReadPubSubOptions(streaming=True)
39 | # Split the source argument into a list of sources that can be read by
40 | # Beam's MultipleReadFromPubSub transform
41 | sources = [PubSubSourceDescriptor(s) for s in options.sources.split(",")]
42 |
43 | with beam.Pipeline(options=options) as p:
44 | (
45 | p
46 | | "Read multiple PubSub sources" >> MultipleReadFromPubSub(sources)
47 | | "Message" >> Map(lambda msg: f"PubSub message:\n{msg}\n")
48 | | Map(logging.info)
49 | )
50 |
51 |
52 | if __name__ == "__main__":
53 | logging.getLogger().setLevel(logging.INFO)
54 | run()
55 |
--------------------------------------------------------------------------------
/Python/pubsub/read_pubsub_subscription.py:
--------------------------------------------------------------------------------
1 | # Copyright 2022 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import Map
21 | from apache_beam.io import ReadFromPubSub
22 | from apache_beam.options.pipeline_options import PipelineOptions
23 |
24 |
25 | def run():
26 | """Run PubSub subscription read function."""
27 |
28 | class ReadPubSubOptions(PipelineOptions):
29 | @classmethod
30 | def _add_argparse_args(cls, parser):
31 | # Add a required flag to allow inputting a subscription from
32 | # the command line
33 | parser.add_argument(
34 | "--subscription",
35 | required=True,
36 | help="PubSub subscription to read.",
37 | )
38 | # Create an instance of our custom options class
39 | options = ReadPubSubOptions(streaming=True)
40 |
41 | # Read from the input PubSub subscription and log the output
42 | with beam.Pipeline(options=options) as p:
43 | (
44 | p
45 | | "Read PubSub subscription"
46 | >> ReadFromPubSub(subscription=options.subscription)
47 | | "Message" >> Map(lambda msg: f"PubSub message:\n{msg}\n")
48 | | Map(logging.info)
49 | )
50 |
51 |
52 | if __name__ == "__main__":
53 | logging.getLogger().setLevel(logging.INFO)
54 | run()
55 |
--------------------------------------------------------------------------------
/Python/pubsub/read_pubsub_topic.py:
--------------------------------------------------------------------------------
1 | # Copyright 2022 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import Map
21 | from apache_beam.io import ReadFromPubSub
22 | from apache_beam.options.pipeline_options import PipelineOptions
23 |
24 |
25 | def run():
26 | class ReadPubSubOptions(PipelineOptions):
27 | @classmethod
28 | def _add_argparse_args(cls, parser):
29 | parser.add_argument(
30 | "--topic",
31 | # Run on Dataflow or authenticate to not get
32 | # 403 PermissionDenied
33 | default="projects/pubsub-public-data/topics/taxirides-realtime",
34 | help="PubSub topic to read",
35 | )
36 |
37 | options = ReadPubSubOptions(streaming=True)
38 |
39 | with beam.Pipeline(options=options) as p:
40 | # When reading from a topic, a new subscription is created.
41 | (
42 | p
43 | | "Read PubSub topic" >> ReadFromPubSub(topic=options.topic)
44 | | "Message" >> Map(lambda msg: f"PubSub message:\n{msg}\n")
45 | | Map(logging.info)
46 | )
47 |
48 |
49 | if __name__ == "__main__":
50 | logging.getLogger().setLevel(logging.INFO)
51 | run()
52 |
--------------------------------------------------------------------------------
/Python/pubsub/read_pubsub_topic_attributes.py:
--------------------------------------------------------------------------------
1 | """Read from a PubSub topic with attributes."""
2 | # Copyright 2022 Google LLC
3 | #
4 | # Licensed under the Apache License, Version 2.0 (the "License");
5 | # you may not use this file except in compliance with the License.
6 | # You may obtain a copy of the License at
7 | #
8 | # https://www.apache.org/licenses/LICENSE-2.0
9 | #
10 | # Unless required by applicable law or agreed to in writing, software
11 | # distributed under the License is distributed on an "AS IS" BASIS,
12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | # See the License for the specific language governing permissions and
14 | # limitations under the License.
15 |
16 | # https://beam.apache.org/releases/pydoc/2.31.0/apache_beam.io.gcp.pubsub.html
17 |
18 | # standard libraries
19 | import logging
20 |
21 | # third party libraries
22 | import apache_beam as beam
23 | from apache_beam import Map
24 | from apache_beam.io import ReadFromPubSub
25 | from apache_beam.options.pipeline_options import PipelineOptions
26 |
27 |
28 | def run():
29 | class ReadPubSubOptions(PipelineOptions):
30 | @classmethod
31 | def _add_argparse_args(cls, parser):
32 | parser.add_argument(
33 | "--topic",
34 | required=True,
35 | help="PubSub topic with attributes to read.",
36 | )
37 |
38 | options = ReadPubSubOptions(streaming=True)
39 |
40 | with beam.Pipeline(options=options) as p:
41 | (
42 | p
43 | | "Read PubSub topic"
44 | >> ReadFromPubSub(topic=options.topic, with_attributes=True)
45 | | "Message"
46 | >> Map(
47 | lambda msg: f"PubSub message:\n"
48 | f"Data: {msg.data}\n"
49 | f"Attributes: {msg.attributes}\n"
50 | )
51 | | Map(logging.info)
52 | )
53 |
54 |
55 | if __name__ == "__main__":
56 | logging.getLogger().setLevel(logging.INFO)
57 | run()
58 |
--------------------------------------------------------------------------------
/Python/pubsub/write_pubsub.py:
--------------------------------------------------------------------------------
1 | # Copyright 2022 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import DoFn, ParDo
21 | from apache_beam.io import ReadFromPubSub, WriteToPubSub
22 | from apache_beam.options.pipeline_options import PipelineOptions
23 |
24 | input_topic = "projects/pubsub-public-data/topics/taxirides-realtime"
25 |
26 |
27 | class PubsubToPubsub(DoFn):
28 | def process(self, element):
29 | # Assuming pubsub input is a byte string
30 | data = element.decode("utf-8")
31 | # Perform some custom transformation here
32 | data = data.encode("utf-8")
33 | yield data
34 |
35 |
36 | def run():
37 | """Write PubSub topic function."""
38 |
39 | class WritePubSubOptions(PipelineOptions):
40 | @classmethod
41 | def _add_argparse_args(cls, parser):
42 | parser.add_argument(
43 | "--topic", required=True, help="PubSub topic to write to."
44 | )
45 |
46 | options = WritePubSubOptions(streaming=True)
47 |
48 | with beam.Pipeline(options=options) as p:
49 | (
50 | p
51 | | "Read from PubSub" >> ReadFromPubSub(topic=input_topic)
52 | | "EncodeString" >> ParDo(PubsubToPubsub())
53 | | "Write to PubSub" >> WriteToPubSub(topic=options.topic)
54 | )
55 |
56 |
57 | if __name__ == "__main__":
58 | logging.getLogger().setLevel(logging.INFO)
59 | run()
60 |
--------------------------------------------------------------------------------
/Python/pubsub/write_pubsub_attributes.py:
--------------------------------------------------------------------------------
1 | # Copyright 2022 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import json
17 | import logging
18 |
19 | # third party libraries
20 | import apache_beam as beam
21 | from apache_beam import Map, ParDo
22 | from apache_beam.io import ReadFromPubSub, WriteToPubSub
23 | from apache_beam.options.pipeline_options import PipelineOptions
24 |
25 | input_topic = "projects/pubsub-public-data/topics/taxirides-realtime"
26 |
27 |
28 | def to_pubsub_message(element):
29 | # third party libraries
30 | from apache_beam.io import PubsubMessage
31 |
32 | if element["ride_status"] == "dropoff":
33 | attributes = {}
34 | attributes["timestamp"] = element["timestamp"]
35 | # Attributes need to be string-string
36 | attributes["passenger_count"] = str(element["passenger_count"])
37 | # Data needs to be string
38 | data = bytes(f"Ride id is {element['ride_id']}", "utf-8")
39 | message = PubsubMessage(data=data, attributes=attributes)
40 | yield message
41 |
42 |
43 | def run():
44 | class WritePubSubAttrOptions(PipelineOptions):
45 | @classmethod
46 | def _add_argparse_args(cls, parser):
47 | parser.add_argument(
48 | "--topic",
49 | required=True,
50 | help="PubSub topic with attributes to write to.",
51 | )
52 |
53 | options = WritePubSubAttrOptions(streaming=True)
54 |
55 | with beam.Pipeline(options=options) as p:
56 | (
57 | p
58 | | "Read PubSub topic" >> ReadFromPubSub(topic=input_topic)
59 | | Map(json.loads)
60 | | "Message" >> ParDo(to_pubsub_message)
61 | | WriteToPubSub(topic=options.topic, with_attributes=True)
62 | )
63 |
64 |
65 | if __name__ == "__main__":
66 | logging.getLogger().setLevel(logging.INFO)
67 | run()
68 |
--------------------------------------------------------------------------------
/Python/pyproject.toml:
--------------------------------------------------------------------------------
1 | # Copyright 2023 Google LLC
2 |
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 |
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 |
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | [tool.black]
16 | line-length = 80
17 | include = '\.pyi?$'
18 | exclude = '''
19 |
20 | (
21 | /(
22 | \.eggs # exclude a few common directories in the
23 | | \.git # root of the project
24 | | \.hg
25 | | \.mypy_cache
26 | | \.tox
27 | | \.vscode
28 | | \.idea
29 | | \.ipynb_checkpoints
30 | | \.dvc
31 | | _build
32 | | buck-out
33 | | build
34 | | dist
35 | | venv
36 | | node_modules
37 | )/
38 | | version.py # also separately exclude a file named foo.py in
39 | # the root of the project
40 | )
41 | '''
--------------------------------------------------------------------------------
/Python/requirements.dev.txt:
--------------------------------------------------------------------------------
1 | # Copyright 2023 Google LLC
2 |
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 |
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 |
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 |
16 | #building
17 | setuptools>=67.7.2
18 |
19 | # format/lint
20 | flake8>=5.0.4
21 | isort>=5.6.4
22 | pre-commit>=2.9.3
23 | black>=22.3.0
24 |
25 | # test
26 | pytest>=6.2.1
27 | pytest-cov>=2.10.1
28 | pytest-ordering
29 | pytest-env
--------------------------------------------------------------------------------
/Python/requirements.txt:
--------------------------------------------------------------------------------
1 | apache-beam[gcp]
2 |
--------------------------------------------------------------------------------
/Python/testing_windows/README.md:
--------------------------------------------------------------------------------
1 | ## Notes
2 |
3 | - Pipelines need to be ran locally. For example:
4 |
5 | ```
6 | python late_data.py
7 | ```
8 |
9 | - At the end of each file there's an explanation of the output
10 |
11 |
--------------------------------------------------------------------------------
/Python/tfrecord/read_tfrecord.py:
--------------------------------------------------------------------------------
1 | # Copyright 2023 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import Map
21 | from apache_beam.io.tfrecordio import ReadFromTFRecord
22 | from apache_beam.options.pipeline_options import PipelineOptions
23 |
24 |
25 | class TFRecordOptions(PipelineOptions):
26 | @classmethod
27 | def _add_argparse_args(cls, parser):
28 | # Add a command line flag to be parsed along
29 | # with other normal PipelineOptions
30 | parser.add_argument(
31 | "--file_pattern",
32 | default="your-file-pattern",
33 | help="A file glob pattern to read TFRecords from."
34 | )
35 |
36 |
37 | def run():
38 | """
39 | This pipeline shows how to read from TFRecord format.
40 | """
41 |
42 | options = TFRecordOptions()
43 |
44 | with beam.Pipeline(options=options) as p:
45 |
46 | output = (
47 | p
48 | | "Read from TFRecord" >> ReadFromTFRecord(
49 | file_pattern=options.file_pattern
50 | )
51 | | "Map from bytes" >> Map(map_from_bytes)
52 | | "Log Data" >> Map(logging.info)
53 | )
54 |
55 |
56 | def map_from_bytes(element):
57 | """
58 | Deserializes the input bytes using pickle library and
59 | returns the reconstructed object.
60 | By default TFRecordIO transforms use `coders.BytesCoder()`.
61 | """
62 | # third party libraries
63 | import pickle
64 |
65 | return pickle.loads(element)
66 |
67 |
68 | if __name__ == "__main__":
69 | logging.getLogger().setLevel(logging.INFO)
70 | run()
71 |
--------------------------------------------------------------------------------
/Python/tfrecord/write_tfrecord.py:
--------------------------------------------------------------------------------
1 | # Copyright 2023 Google LLC
2 | #
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 | #
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 | #
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | # standard libraries
16 | import logging
17 |
18 | # third party libraries
19 | import apache_beam as beam
20 | from apache_beam import Create, Map
21 | from apache_beam.io.tfrecordio import WriteToTFRecord
22 | from apache_beam.options.pipeline_options import PipelineOptions
23 |
24 |
25 | class TFRecordOptions(PipelineOptions):
26 | @classmethod
27 | def _add_argparse_args(cls, parser):
28 | # Add a command line flag to be parsed along
29 | # with other normal PipelineOptions
30 | parser.add_argument(
31 | "--file_path_prefix",
32 | default="your-file-path-prefix",
33 | help="A file path prefix to write TFRecords files to."
34 | )
35 |
36 |
37 | def run():
38 | """
39 | This pipeline shows how to write to TFRecord format.
40 | """
41 |
42 | options = TFRecordOptions()
43 |
44 | with beam.Pipeline(options=options) as p:
45 |
46 | elements = [
47 | (1, "Charles"),
48 | (2, "Alice"),
49 | (3, "Bob"),
50 | (4, "Amanda"),
51 | (5, "Alex"),
52 | (6, "Eliza")
53 | ]
54 |
55 | output = (
56 | p
57 | | "Create" >> Create(elements)
58 | | "Map to Bytes" >> Map(map_to_bytes)
59 | | "Write to TFRecord" >> WriteToTFRecord(
60 | file_path_prefix=options.file_path_prefix
61 | )
62 | )
63 |
64 |
65 | def map_to_bytes(element):
66 | """
67 | Serializes the input element using pickle library and
68 | returns the bytes representation.
69 | By default TFRecordIO transforms use `coders.BytesCoder()`.
70 | """
71 | # third party libraries
72 | import pickle
73 |
74 | return pickle.dumps(element)
75 |
76 |
77 | if __name__ == "__main__":
78 | logging.getLogger().setLevel(logging.INFO)
79 | run()
80 |
--------------------------------------------------------------------------------
/Python/yaml/README.md:
--------------------------------------------------------------------------------
1 | ## Running pipelines
2 |
3 | The Beam yaml parser is currently included as part of the Apache Beam Python SDK.
4 | This can be installed (e.g. within a virtual environment) as
5 |
6 | ```
7 | pip install apache_beam[yaml,gcp]
8 | ```
9 |
10 | In addition, several of the provided transforms (such as SQL) are implemented
11 | in Java and their expansion will require a working Java interpeter. (The
12 | requisite artifacts will be automatically downloaded from the apache maven
13 | repositories, so no further installs will be required.)
14 | Docker is also currently required for local execution of these
15 | cross-language-requiring transforms, but not for submission to a non-local
16 | runner such as Flink or Dataflow.
17 |
18 | Once the prerequisites are installed, you can execute a pipeline defined
19 | in a yaml file as
20 |
21 | ```
22 | python -m apache_beam.yaml.main --yaml_pipeline_file=/path/to/pipeline.yaml [other pipeline options such as the runner]
23 | ```
--------------------------------------------------------------------------------
/Python/yaml/bigquery_to_kafka.yaml:
--------------------------------------------------------------------------------
1 | # Copyright 2024 Google LLC
2 |
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 |
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 |
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | pipeline:
16 | type: chain
17 |
18 | source:
19 | type: ReadFromBigQuery
20 | config:
21 | table: project-id.dataset-id.table-id
22 |
23 | transforms:
24 | - type: Filter
25 | config:
26 | language: python
27 | keep: "id > 2"
28 | - type: MapToFields
29 | config:
30 | language: python
31 | fields:
32 | id:
33 | callable: "lambda row: row.id"
34 | name:
35 | callable: "lambda row: row.name.upper()"
36 |
37 | sink:
38 | type: WriteToKafka
39 | config:
40 | bootstrapServers: localhost:9092
41 | topic: topic-name
42 | format: JSON
43 |
--------------------------------------------------------------------------------
/Python/yaml/kafka_to_bigquery.yaml:
--------------------------------------------------------------------------------
1 | # Copyright 2024 Google LLC
2 |
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 |
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 |
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | pipeline:
16 | type: chain
17 |
18 | source:
19 | type: ReadFromKafka
20 | config:
21 | bootstrapServers: localhost:9092
22 | topic: topic-name
23 | format: RAW
24 |
25 | transforms:
26 | - type: MapToFields
27 | config:
28 | language: python
29 | fields:
30 | message:
31 | callable: "lambda row: str(row.payload.decode('utf-8'))"
32 |
33 | sink:
34 | type: WriteToBigQuery
35 | config:
36 | table: project-id.dataset-id.table-id
37 | create_disposition: CREATE_IF_NEEDED
38 | write_disposition: WRITE_APPEND
39 | windowing:
40 | type: fixed
41 | size: 60s
42 |
43 | options:
44 | streaming: true
45 | runner: DataflowRunner
46 | project: project-id
47 | region: us-central1
48 | temp_location: gs://path/to/temp/folder
49 |
--------------------------------------------------------------------------------
/Python/yaml/kafka_to_json.yaml:
--------------------------------------------------------------------------------
1 | # Copyright 2024 Google LLC
2 |
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 |
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 |
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | pipeline:
16 | type: chain
17 |
18 | source:
19 | type: ReadFromKafka
20 | config:
21 | bootstrapServers: localhost:9092
22 | topic: topic-name
23 | format: RAW
24 |
25 | transforms:
26 | - type: MapToFields
27 | config:
28 | language: python
29 | fields:
30 | message:
31 | callable: "lambda row: row.payload.decode('utf-8')"
32 |
33 | sink:
34 | type: WriteToJson
35 | config:
36 | path: gs://path/to/output.json
37 | windowing:
38 | type: fixed
39 | size: 60s
40 |
41 | options:
42 | streaming: true
43 | runner: DataflowRunner
44 | project: project-id
45 | region: us-central1
46 | temp_location: gs://path/to/temp/folder
47 |
--------------------------------------------------------------------------------
/Python/yaml/test_csv_to_json.yaml:
--------------------------------------------------------------------------------
1 | # Copyright 2024 Google LLC
2 |
3 | # Licensed under the Apache License, Version 2.0 (the "License");
4 | # you may not use this file except in compliance with the License.
5 | # You may obtain a copy of the License at
6 |
7 | # https://www.apache.org/licenses/LICENSE-2.0
8 |
9 | # Unless required by applicable law or agreed to in writing, software
10 | # distributed under the License is distributed on an "AS IS" BASIS,
11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | # See the License for the specific language governing permissions and
13 | # limitations under the License.
14 |
15 | pipeline:
16 | transforms:
17 | - type: ReadFromCsv
18 | config:
19 | path: /path/to/input*.csv
20 | - type: Filter
21 | config:
22 | language: python
23 | keep: "col2 > 100"
24 | input: ReadFromCsv
25 | - type: Sql
26 | config:
27 | query: "select col1, count(*) as cnt from PCOLLECTION group by col1"
28 | input: Filter
29 | - type: WriteToJson
30 | config:
31 | path: /path/to/output.json
32 | input: Sql
33 |
--------------------------------------------------------------------------------
/Scala/.gitignore:
--------------------------------------------------------------------------------
1 | *.class
2 | *.log
3 |
4 | # virtual machine crash logs, see http://www.java.com/en/download/help/error_hotspot.xml
5 | hs_err_pid*
6 |
7 |
--------------------------------------------------------------------------------
/Scala/README.md:
--------------------------------------------------------------------------------
1 | ## SCio
2 |
3 | **SCio** is a wrap up of *Apache Beam* for **Scala**, made by Spotify.
4 |
5 | In order to use this cookbook, you need to have Scala in your computer.
6 |
7 | To launch the pipelines, run commands like this from the root:
8 |
9 | `sbt "runMain basics.Join"`
10 |
11 | If you want to use Dataflow, you need to add your [credentials](https://cloud.google.com/docs/authentication/getting-started)
12 | and run:
13 |
14 | ```
15 | export BUCKET=
16 | export REGION=
17 | export PROJECT=
18 | sbt "runMain basics.Join --project=$PROJECT --region=$REGION --runner=DataflowRunner --tempLocation=gs://$BUCKET/tmp/"
19 | ```
20 |
21 | Find more documentation here:
22 |
23 | - https://spotify.github.io/scio/index.html
24 | - https://github.com/spotify/scio
--------------------------------------------------------------------------------
/Scala/src/main/scala/basics/ApplyTransform.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2022 Google LLC
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * https://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package basics
18 |
19 | import com.spotify.scio.{Args, ContextAndArgs, ScioContext}
20 | import org.apache.beam.sdk.transforms.{Count, PTransform, Reshuffle}
21 | import org.apache.beam.sdk.values._
22 | import org.slf4j.LoggerFactory
23 |
24 | object ApplyTransform {
25 |
26 | private val log = LoggerFactory.getLogger(this.getClass)
27 |
28 | // Beam transform
29 | val countPerKey: PTransform[PCollection[KV[Int, Int]], PCollection[KV[Int, java.lang.Long]]] = Count.perKey()
30 |
31 | def main(cmdlineArgs: Array[String]): Unit = {
32 | val (scontext: ScioContext, opts: Args) = ContextAndArgs(cmdlineArgs)
33 | implicit val sc: ScioContext = scontext
34 |
35 | val elements = sc
36 | .parallelize(1 to 100)
37 | // Beam transform
38 | .applyTransform(Reshuffle.viaRandomKey())
39 |
40 | elements
41 | .map { x: Int => KV.of(x % 2, x) }
42 | .withName("applyTransform CountPerKey")
43 | .applyTransform(countPerKey)
44 | .map(kv => log.info(s"key ${kv.getKey()} has ${kv.getValue()} elements"))
45 |
46 | sc.run()
47 | }
48 | }
49 |
--------------------------------------------------------------------------------
/Scala/src/main/scala/basics/CoGroupByKey.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2022 Google LLC
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * https://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package basics
18 |
19 | import com.spotify.scio.values.SCollection
20 | import com.spotify.scio.{Args, ContextAndArgs, ScioContext}
21 | import org.slf4j.LoggerFactory
22 |
23 |
24 | object CoGroupByKey {
25 |
26 | private val log = LoggerFactory.getLogger(this.getClass)
27 |
28 | def main(cmdlineArgs: Array[String]): Unit = {
29 | val (scontext: ScioContext, opts: Args) = ContextAndArgs(cmdlineArgs)
30 | implicit val sc: ScioContext = scontext
31 |
32 | val jobs = Seq(
33 | ("Anna", "SWE"),
34 | ("Kim", "Data Engineer"),
35 | ("Kim", "Data Scientist"),
36 | ("Robert", "Artist"),
37 | ("Sophia", "CEO"),
38 | ("Ruben", "Writer")
39 | )
40 |
41 | val hobbies = Seq(
42 | ("Anna", "Painting"),
43 | ("Kim", "Football"),
44 | ("Kim", "Gardening"),
45 | ("Robert", "Swimming"),
46 | ("Sophia", "Mathematics"),
47 | ("Sophia", "Tennis")
48 | )
49 |
50 | val createJobs: SCollection[(String, String)] = sc.parallelize(jobs)
51 | val createHobbies: SCollection[(String, String)] = sc.parallelize(hobbies)
52 |
53 | createJobs
54 | .cogroup(createHobbies)
55 | .map { kv => (kv._1, s"Jobs: ${kv._2._1.mkString(", ")} and Hobbies: ${kv._2._2.mkString(", ")}") }
56 | .withName("log")
57 | .map { kv => log.info(s"Key: ${kv._1}; Cogrouped-Value: ${kv._2}") }
58 |
59 | sc.run()
60 | }
61 | }
62 |
--------------------------------------------------------------------------------
/Scala/src/main/scala/basics/CombineGlobally.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2022 Google LLC
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * https://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package basics
18 |
19 | import com.spotify.scio.values.SCollection
20 | import com.spotify.scio.{Args, ContextAndArgs, ScioContext}
21 | import org.slf4j.LoggerFactory
22 |
23 |
24 | object CombineGlobally {
25 |
26 | private val log = LoggerFactory.getLogger(this.getClass)
27 |
28 | def main(cmdlineArgs: Array[String]): Unit = {
29 | val (scontext: ScioContext, opts: Args) = ContextAndArgs(cmdlineArgs)
30 | implicit val sc: ScioContext = scontext
31 |
32 | val create: SCollection[Int] = sc.parallelize(1 to 10)
33 |
34 | create
35 | // reduce applies a simple operation to the elements as a group (Combine)
36 | .reduce(_ + _)
37 | .withName("log")
38 | .map { x => log.info("Sum is " + x) }
39 |
40 |
41 | sc.run()
42 | }
43 | }
44 |
--------------------------------------------------------------------------------
/Scala/src/main/scala/basics/CombineGloballyInterface.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2022 Google LLC
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * https://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package basics
18 |
19 | import com.spotify.scio.values.SCollection
20 | import com.spotify.scio.{Args, ContextAndArgs, ScioContext}
21 | import org.slf4j.LoggerFactory
22 |
23 |
24 | object CombineGloballyInterface {
25 |
26 | private val log = LoggerFactory.getLogger(this.getClass)
27 |
28 | def main(cmdlineArgs: Array[String]): Unit = {
29 | val (scontext: ScioContext, opts: Args) = ContextAndArgs(cmdlineArgs)
30 | implicit val sc: ScioContext = scontext
31 |
32 | val create: SCollection[Int] = sc.parallelize(1 to 10)
33 |
34 | create
35 | // Calculate avg
36 | // 0.0, 0 is the first accumulator
37 | .aggregate(0.0, 0)(
38 | (acc, x) => (acc._1 + x, acc._2 + 1), // add input
39 | (l, r) => (l._1 + r._1, l._2 + r._2) // merge accumulators, type kv
40 | )
41 | .map { kv => kv._1 / kv._2 }
42 | .withName("log")
43 | .map { x => log.info("Avg is " + x) }
44 |
45 |
46 | sc.run()
47 | }
48 | }
49 |
--------------------------------------------------------------------------------
/Scala/src/main/scala/basics/CombinePerKey.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2022 Google LLC
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * https://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package basics
18 |
19 | import com.spotify.scio.values.SCollection
20 | import com.spotify.scio.{Args, ContextAndArgs, ScioContext}
21 | import org.slf4j.LoggerFactory
22 |
23 |
24 | object CombinePerKey {
25 |
26 | private val log = LoggerFactory.getLogger(this.getClass)
27 |
28 | def main(cmdlineArgs: Array[String]): Unit = {
29 | val (scontext: ScioContext, opts: Args) = ContextAndArgs(cmdlineArgs)
30 | implicit val sc: ScioContext = scontext
31 |
32 | val create: SCollection[Int] = sc.parallelize(1 to 100)
33 |
34 | create
35 | .withName("calculateDivisors")
36 | .flatMap { n => (1 to n).filter { x => n % x == 0 }.map { x => (n, x) } }
37 | // reduceByKey applies a simple combine operation to the values of a KV
38 | .reduceByKey(_ + _)
39 | .map { kv =>
40 | val isPerfect = kv._2.toFloat / kv._1 match {
41 | case 2 => "is a perfect number "
42 | case _ => "is not a perfect number"
43 | }
44 | log.info(s"${kv._1} $isPerfect")
45 | }
46 |
47 |
48 | sc.run()
49 | }
50 | }
51 |
--------------------------------------------------------------------------------
/Scala/src/main/scala/basics/CombinePerKeyInterface.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2022 Google LLC
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * https://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package basics
18 |
19 | import com.spotify.scio.values.SCollection
20 | import com.spotify.scio.{Args, ContextAndArgs, ScioContext}
21 | import org.slf4j.LoggerFactory
22 |
23 |
24 | object CombinePerKeyInterface {
25 |
26 | private val log = LoggerFactory.getLogger(this.getClass)
27 |
28 | def main(cmdlineArgs: Array[String]): Unit = {
29 | val (scontext: ScioContext, opts: Args) = ContextAndArgs(cmdlineArgs)
30 | implicit val sc: ScioContext = scontext
31 |
32 | val create: SCollection[Int] = sc.parallelize(1 to 30)
33 |
34 | create
35 | .withName("calculateDivisors")
36 | .flatMap { n => (1 to n).filter { x => n % x == 0 }.map { x => (n, x) } }
37 | // first _ + _ is adding the values, second _ + _ is adding the accumulators
38 | // 0.0 is the first accumulator
39 | .aggregateByKey(0.0)(_ + _, _ + _)
40 | .map { kv =>
41 | val isPerfect = kv._2.toFloat / kv._1 match {
42 | case 2 => "is a perfect number "
43 | case _ => "is not a perfect number"
44 | }
45 | log.info(s"${kv._1} $isPerfect")
46 | }
47 |
48 |
49 | sc.run()
50 | }
51 | }
52 |
--------------------------------------------------------------------------------
/Scala/src/main/scala/basics/CreateInteger.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2022 Google LLC
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * https://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package basics
18 |
19 | import com.spotify.scio.values.SCollection
20 | import com.spotify.scio.{Args, ContextAndArgs, ScioContext}
21 | import org.slf4j.LoggerFactory
22 |
23 |
24 | object CreateInteger {
25 |
26 | private val log = LoggerFactory.getLogger(this.getClass)
27 |
28 | def main(cmdlineArgs: Array[String]): Unit = {
29 | val (scontext: ScioContext, opts: Args) = ContextAndArgs(cmdlineArgs)
30 | implicit val sc: ScioContext = scontext
31 |
32 | // this creates integers from 1 to 100. Same as Create in Beam
33 | val create: SCollection[Int] = sc.parallelize(1 to 100)
34 |
35 | create.map { x => log.info(x.toString) }
36 |
37 | sc.run()
38 | }
39 | }
40 |
--------------------------------------------------------------------------------
/Scala/src/main/scala/basics/CreateIterator.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2022 Google LLC
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * https://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package basics
18 |
19 | import com.spotify.scio.values.SCollection
20 | import com.spotify.scio.{Args, ContextAndArgs, ScioContext}
21 | import org.slf4j.LoggerFactory
22 |
23 |
24 | object CreateIterator {
25 |
26 | private val log = LoggerFactory.getLogger(this.getClass)
27 |
28 | def main(cmdlineArgs: Array[String]): Unit = {
29 | val (scontext: ScioContext, opts: Args) = ContextAndArgs(cmdlineArgs)
30 | implicit val sc: ScioContext = scontext
31 |
32 | // generate iterator
33 | val elements = Seq(("Carlos", 23), ("Maria", 27), ("Ruben", 15), ("Juana", 38), ("Kim", 77))
34 |
35 | // this creates the iterator from above. Same as Create in Beam
36 | val create: SCollection[(String, Int)] = sc.parallelize(elements)
37 |
38 | create.map { kv => log.info("Key: " + kv._1 + ", Value: " + kv._2) }
39 |
40 |
41 | sc.run()
42 | }
43 | }
44 |
--------------------------------------------------------------------------------
/Scala/src/main/scala/basics/Flatten.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2022 Google LLC
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * https://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package basics
18 |
19 | import com.spotify.scio.values.SCollection
20 | import com.spotify.scio.{Args, ContextAndArgs, ScioContext}
21 | import org.slf4j.LoggerFactory
22 |
23 |
24 | object Flatten {
25 |
26 | private val log = LoggerFactory.getLogger(this.getClass)
27 |
28 | def main(cmdlineArgs: Array[String]): Unit = {
29 | val (scontext: ScioContext, opts: Args) = ContextAndArgs(cmdlineArgs)
30 | implicit val sc: ScioContext = scontext
31 |
32 | val first: SCollection[Int] = sc.parallelize(1 to 100)
33 | val second: SCollection[Int] = sc.parallelize(101 to 200)
34 |
35 |
36 | // Beam Flatten in SCio is done with operation `++`.
37 | // Do not mistake the Flatten from Beam with the pure flatten from SCio. Check SCioFlatten for example
38 | first
39 | .++ {
40 | second
41 | }
42 | .withName("log")
43 | .map { x => log.info(x.toString) }
44 |
45 | sc.run()
46 | }
47 | }
48 |
--------------------------------------------------------------------------------
/Scala/src/main/scala/basics/GroupByKey.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2022 Google LLC
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * https://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package basics
18 |
19 | import com.spotify.scio.values.SCollection
20 | import com.spotify.scio.{Args, ContextAndArgs, ScioContext}
21 | import org.slf4j.LoggerFactory
22 |
23 |
24 | object GroupByKey {
25 |
26 | private val log = LoggerFactory.getLogger(this.getClass)
27 |
28 | def main(cmdlineArgs: Array[String]): Unit = {
29 | val (scontext: ScioContext, opts: Args) = ContextAndArgs(cmdlineArgs)
30 | implicit val sc: ScioContext = scontext
31 |
32 | // generate iterator
33 | val elements = Seq(
34 | ("Mammal", "Dog"),
35 | ("Mammal", "Cat"),
36 | ("Fish", "Salmon"),
37 | ("Amphibian", "Snake"),
38 | ("Bird", "Eagle"),
39 | ("Bird", "Owl")
40 | )
41 |
42 | val create: SCollection[(String, String)] = sc.parallelize(elements)
43 |
44 | create
45 | .groupByKey
46 | .map { kv => (kv._1, kv._2.mkString(", ")) }
47 | .withName("log")
48 | .map { kv => log.info("Key: " + kv._1 + "; Value: " + kv._2) }
49 | sc.run()
50 | }
51 | }
52 |
--------------------------------------------------------------------------------
/Scala/src/main/scala/basics/ListSideInput.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2022 Google LLC
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * https://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package basics
18 |
19 | import com.spotify.scio.values.SCollection
20 | import com.spotify.scio.{Args, ContextAndArgs, ScioContext}
21 | import org.slf4j.LoggerFactory
22 |
23 |
24 | object ListSideInput {
25 |
26 | private val log = LoggerFactory.getLogger(this.getClass)
27 |
28 | def main(cmdlineArgs: Array[String]): Unit = {
29 | val (scontext: ScioContext, opts: Args) = ContextAndArgs(cmdlineArgs)
30 | implicit val sc: ScioContext = scontext
31 |
32 | val elements = Seq(
33 | ("USD", 3.1415),
34 | ("USD", 1729.0),
35 | ("CHF", 2.7182),
36 | ("EUR", 1.618),
37 | ("CHF", 1.1),
38 | ("CHF", 342.45),
39 | ("EUR", 890.01)
40 | )
41 |
42 | val rates = Seq(
43 | ("USD", 1.0),
44 | ("EUR", 0.8),
45 | ("CHF", 0.9)
46 | )
47 |
48 | val values: SCollection[(String, Double)] = sc.parallelize(elements)
49 | val ratesSCol: SCollection[(String, Double)] = sc.parallelize(rates)
50 |
51 | val rateSide = ratesSCol.asListSideInput // Creates a SideInput as List
52 |
53 | values
54 | .withSideInputs(rateSide) // point to side input (List)
55 | .map { (kv, s) =>
56 | // kv is the main value, s the side input
57 | // to retrieve the side input we need, we do s(SideInput Variable)
58 | val rate: Double = s(rateSide).filter { r => r._1.equals(kv._1) }(0)._2
59 | val changed = s"${kv._2} ${kv._1} are ${kv._2 * rate} USD"
60 | changed
61 | }
62 | .toSCollection // This is needed so that the Side Input is not kept in the next operation
63 | .withName("log")
64 | .map { s: String => log.info(s) }
65 |
66 |
67 | sc.run()
68 | }
69 | }
70 |
--------------------------------------------------------------------------------
/Scala/src/main/scala/basics/MapSideInput.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2022 Google LLC
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * https://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package basics
18 |
19 | import com.spotify.scio.values.SCollection
20 | import com.spotify.scio.{Args, ContextAndArgs, ScioContext}
21 | import org.slf4j.LoggerFactory
22 |
23 |
24 | object MapSideInput {
25 |
26 | private val log = LoggerFactory.getLogger(this.getClass)
27 |
28 | def main(cmdlineArgs: Array[String]): Unit = {
29 | val (scontext: ScioContext, opts: Args) = ContextAndArgs(cmdlineArgs)
30 | implicit val sc: ScioContext = scontext
31 |
32 | val elements = Seq(
33 | ("USD", 3.1415),
34 | ("USD", 1729.0),
35 | ("CHF", 2.7182),
36 | ("EUR", 1.618),
37 | ("CHF", 1.1),
38 | ("CHF", 342.45),
39 | ("EUR", 890.01)
40 | )
41 |
42 | val rates = Seq(
43 | ("USD", 1.0),
44 | ("EUR", 0.8),
45 | ("CHF", 0.9)
46 | )
47 |
48 | val values: SCollection[(String, Double)] = sc.parallelize(elements)
49 | val ratesSCol: SCollection[(String, Double)] = sc.parallelize(rates)
50 |
51 | val rateSide = ratesSCol.asMapSideInput // Creates a SideInput as Map
52 |
53 | values
54 | .withSideInputs(rateSide) // point to side input (Map)
55 | .map { (kv, s) =>
56 | // kv is the main value, s the side input
57 | // to retrieve the side input we need, we do s(SideInput Variable)
58 | val rate: Double = s(rateSide).getOrElse(kv._1, 1.0)
59 | val changed = s"${kv._2} ${kv._1} are ${kv._2 * rate} USD"
60 | changed
61 | }
62 | .toSCollection // This is needed so that the Side Input is not kept in the next operation
63 | .withName("log")
64 | .map { s: String => log.info(s) }.withName("log")
65 |
66 |
67 | sc.run()
68 | }
69 | }
70 |
--------------------------------------------------------------------------------
/Scala/src/main/scala/basics/MultipleOutputs.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2022 Google LLC
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * https://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package basics
18 |
19 | import com.spotify.scio.values.SideOutput
20 | import com.spotify.scio.{Args, ContextAndArgs, ScioContext}
21 | import org.slf4j.LoggerFactory
22 |
23 |
24 | object MultipleOutputs {
25 |
26 | private val log = LoggerFactory.getLogger(this.getClass)
27 |
28 | def main(cmdlineArgs: Array[String]): Unit = {
29 | val (scontext: ScioContext, opts: Args) = ContextAndArgs(cmdlineArgs)
30 | implicit val sc: ScioContext = scontext
31 |
32 | // define side outputs
33 | val multiplesThree = SideOutput[Int]()
34 | val multiplesFive = SideOutput[Int]()
35 | val multipleBoth = SideOutput[Int]()
36 |
37 | val (main, outputs) = sc
38 | .parallelize(1 to 30)
39 | // list outputs
40 | .withSideOutputs(multiplesThree, multiplesFive, multipleBoth)
41 | // n is the element, c the context
42 | .flatMap { (n, c) =>
43 | (n % 3, n % 5) match {
44 | case (0, 0) => c.output(multipleBoth, n); c.output(multiplesFive, n); c.output(multiplesThree, n)
45 | case (_, 0) => c.output(multiplesFive, n)
46 | case (0, _) => c.output(multiplesThree, n)
47 | case _ =>
48 | }
49 | // main output
50 | Some(n)
51 | }
52 |
53 | main.map { i => log.info(s"Main value $i") }
54 |
55 | outputs(multiplesThree).map { i => log.info(s"Multiple of three: $i") }
56 | outputs(multiplesFive).map { i => log.info(s"Multiple of five: $i") }
57 | outputs(multipleBoth).map { i => log.info(s"Multiple of both: $i") }
58 |
59 |
60 | sc.run()
61 | }
62 | }
63 |
--------------------------------------------------------------------------------
/Scala/src/main/scala/basics/PardoFns.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2022 Google LLC
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * https://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package basics
18 |
19 | import com.spotify.scio.values.SCollection
20 | import com.spotify.scio.{Args, ContextAndArgs, ScioContext}
21 | import org.slf4j.LoggerFactory
22 |
23 |
24 | object PardoFns {
25 |
26 | private val log = LoggerFactory.getLogger(this.getClass)
27 |
28 | def main(cmdlineArgs: Array[String]): Unit = {
29 | val (scontext: ScioContext, opts: Args) = ContextAndArgs(cmdlineArgs)
30 | implicit val sc: ScioContext = scontext
31 |
32 | val create: SCollection[Int] = sc.parallelize(1 to 10)
33 |
34 | // SCio overrides ParDo Fns into scala natives flatmap and map
35 | create
36 | // flatMap calculates the divisors of a number
37 | .flatMap { n => (1 to n).filter { x => n % x == 0 }.map { x => (n, x) } }
38 | .map { kv => "Number " + kv._1 + " has divisor " + kv._2 }
39 | .map {
40 | log.info(_)
41 | }
42 |
43 | sc.run()
44 | }
45 | }
46 |
--------------------------------------------------------------------------------
/Scala/src/main/scala/basics/ScioFlatten.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2022 Google LLC
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * https://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package basics
18 |
19 | import com.spotify.scio.values.SCollection
20 | import com.spotify.scio.{Args, ContextAndArgs, ScioContext}
21 | import org.slf4j.LoggerFactory
22 |
23 | // NOTE: THIS IS NOT THE TYPICAL BEAM FLATTEN, this is the equivalen of Flatten.iterables
24 | // check file Flatten for that
25 | object ScioFlatten {
26 |
27 | private val log = LoggerFactory.getLogger(this.getClass)
28 |
29 | def main(cmdlineArgs: Array[String]): Unit = {
30 | val (scontext: ScioContext, opts: Args) = ContextAndArgs(cmdlineArgs)
31 | implicit val sc: ScioContext = scontext
32 |
33 | // generate 4 elements that are sequences
34 | val elements = Seq(
35 | Seq("Carlos", "Maria", "Ruben"),
36 | Seq("Car", "Bike", "Plane", "Boat", "Hovercraft"),
37 | Seq("Dog", "Cat"),
38 | Seq("Dataflow")
39 | )
40 |
41 | val create: SCollection[Seq[String]] = sc.parallelize(elements)
42 |
43 | // flatten takes Seq of objects and send its values once by one as single elements
44 | // same as FlattenIterables
45 | create
46 | .flatten
47 | .withName("log")
48 | .map {
49 | log.info(_)
50 | }
51 |
52 |
53 | sc.run()
54 | }
55 | }
56 |
--------------------------------------------------------------------------------
/Scala/src/main/scala/bigquery/ReadQuery.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2022 Google LLC
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * https://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package bigquery
18 |
19 | import com.spotify.scio.bigquery._
20 | import com.spotify.scio.{Args, ContextAndArgs, ScioContext}
21 | import org.slf4j.LoggerFactory
22 |
23 |
24 | object ReadQuery {
25 |
26 | private val log = LoggerFactory.getLogger(this.getClass)
27 |
28 | def main(cmdlineArgs: Array[String]): Unit = {
29 | val (scontext: ScioContext, opts: Args) = ContextAndArgs(cmdlineArgs)
30 | implicit val sc: ScioContext = scontext
31 |
32 | // pipeline option parameters
33 | val sqlDef = "SELECT station_id, status FROM `bigquery-public-data.austin_bikeshare.bikeshare_stations`"
34 | val query: String = opts.getOrElse("query", sqlDef)
35 |
36 | val bqRead = sc.bigQuerySelect(Query(query))
37 |
38 | bqRead
39 | .map {
40 | extractRow
41 | }
42 | .map {
43 | log.info(_)
44 | }
45 |
46 | def extractRow(row: TableRow): String = {
47 | val id = row.getLong("station_id")
48 | val status = row.getString("status")
49 | s"Bike station $id is $status"
50 | }
51 |
52 | sc.run()
53 | }
54 |
55 |
56 | }
57 |
--------------------------------------------------------------------------------
/Scala/src/main/scala/bigquery/ReadStorageAPI.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2022 Google LLC
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * https://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package bigquery
18 |
19 | import com.spotify.scio.bigquery._
20 | import com.spotify.scio.{Args, ContextAndArgs, ScioContext}
21 | import org.slf4j.LoggerFactory
22 |
23 |
24 | object ReadStorageAPI {
25 |
26 | private val log = LoggerFactory.getLogger(this.getClass)
27 |
28 | def main(cmdlineArgs: Array[String]): Unit = {
29 | val (scontext: ScioContext, opts: Args) = ContextAndArgs(cmdlineArgs)
30 | implicit val sc: ScioContext = scontext
31 |
32 | // pipeline option parameters
33 | val tableOp: String = opts.getOrElse("table", "bigquery-public-data:austin_bikeshare.bikeshare_stations")
34 |
35 | val table = Table.Spec(tableOp)
36 |
37 | val bqRead = sc.bigQueryStorage(
38 | table,
39 | selectedFields = List("station_id", "status"),
40 | rowRestriction = "status = \"active\""
41 | )
42 |
43 | bqRead
44 | .map {
45 | extractRow
46 | }
47 | .map {
48 | log.info(_)
49 | }
50 |
51 | def extractRow(row: TableRow): String = {
52 | val id = row.getLong("station_id")
53 | val status = row.getString("status")
54 | s"Bike station $id is $status"
55 | }
56 |
57 | sc.run()
58 | }
59 |
60 |
61 | }
62 |
--------------------------------------------------------------------------------
/Scala/src/main/scala/bigquery/ReadTable.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2022 Google LLC
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * https://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package bigquery
18 |
19 | import com.spotify.scio.bigquery._
20 | import com.spotify.scio.{Args, ContextAndArgs, ScioContext}
21 | import org.slf4j.LoggerFactory
22 |
23 |
24 | object ReadTable {
25 |
26 | private val log = LoggerFactory.getLogger(this.getClass)
27 |
28 | def main(cmdlineArgs: Array[String]): Unit = {
29 | val (scontext: ScioContext, opts: Args) = ContextAndArgs(cmdlineArgs)
30 | implicit val sc: ScioContext = scontext
31 |
32 | // pipeline option parameters
33 | val table: String = opts.getOrElse("table", "bigquery-public-data:austin_bikeshare.bikeshare_stations")
34 |
35 | val bqRead = sc.bigQueryTable(Table.Spec(table))
36 |
37 | bqRead
38 | .map {
39 | extractRow
40 | }
41 | .map {
42 | log.info(_)
43 | }
44 |
45 | def extractRow(row: TableRow): String = {
46 | val id = row.getLong("station_id")
47 | val status = row.getString("status")
48 | s"Bike station $id is $status"
49 | }
50 |
51 | sc.run()
52 | }
53 |
54 |
55 | }
56 |
--------------------------------------------------------------------------------
/Scala/src/main/scala/bigquery/ReadTypedTable.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2022 Google LLC
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * https://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package bigquery
18 |
19 | import com.spotify.scio.bigquery._
20 | import com.spotify.scio.{Args, ContextAndArgs, ScioContext}
21 | import org.slf4j.LoggerFactory
22 |
23 |
24 | object ReadTypedTable {
25 |
26 | private val log = LoggerFactory.getLogger(this.getClass)
27 |
28 | def main(cmdlineArgs: Array[String]): Unit = {
29 | val (scontext: ScioContext, opts: Args) = ContextAndArgs(cmdlineArgs)
30 | implicit val sc: ScioContext = scontext
31 |
32 | // case class for bike station
33 | case class BikeStation(station_id: Long, status: String)
34 |
35 | // pipeline option parameters
36 | val table: String = opts.getOrElse("table", "bigquery-public-data:austin_bikeshare.bikeshare_stations")
37 |
38 | val bqRead = sc.typedBigQueryTable[BikeStation](Table.Spec(table))
39 |
40 | bqRead
41 | .map { s: BikeStation => log.info(s"Bike station ${s.station_id} is ${s.status}") }
42 |
43 | sc.run()
44 | }
45 |
46 |
47 | }
48 |
--------------------------------------------------------------------------------
/Scala/src/main/scala/bigquery/WriteBatch.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2022 Google LLC
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * https://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package bigquery
18 |
19 | import com.spotify.scio.bigquery._
20 | import com.spotify.scio.values.SCollection
21 | import com.spotify.scio.{Args, ContextAndArgs, ScioContext}
22 | import org.slf4j.LoggerFactory
23 |
24 |
25 | object WriteBatch {
26 |
27 | private val log = LoggerFactory.getLogger(this.getClass)
28 |
29 | // Define data class to write to BQ, schema is inferred
30 | @BigQueryType.toTable
31 | case class Person(
32 | name: String,
33 | age: Int,
34 | country: String
35 | )
36 |
37 | def main(cmdlineArgs: Array[String]): Unit = {
38 | val (scontext: ScioContext, opts: Args) = ContextAndArgs(cmdlineArgs)
39 | implicit val sc: ScioContext = scontext
40 |
41 | // pipeline option parameters
42 | val table: String = opts("table")
43 |
44 | val elements = Seq(
45 | ("Carlos", 23, "Spain"),
46 | ("Maria", 27, "UK"),
47 | ("Ruben", 15, "Japan"),
48 | ("Juana", 38, "Ghana"),
49 | ("Kim", 77, "Brazil")
50 | )
51 |
52 | val create: SCollection[(String, Int, String)] = sc.parallelize(elements)
53 |
54 | create
55 | .map[Person] { row: (String, Int, String) => Person(row._1, row._2, row._3) }
56 | .saveAsBigQueryTable(Table.Spec(table), WRITE_APPEND, CREATE_IF_NEEDED)
57 |
58 | sc.run()
59 | }
60 |
61 |
62 | }
63 |
--------------------------------------------------------------------------------
/Scala/src/main/scala/bigquery/WriteDynamicTable.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2022 Google LLC
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * https://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package bigquery
18 |
19 | import com.spotify.scio.bigquery._
20 | import com.spotify.scio.bigquery.dynamic._
21 | import com.spotify.scio.values.SCollection
22 | import com.spotify.scio.{Args, ContextAndArgs, ScioContext}
23 | import org.apache.beam.sdk.io.gcp.bigquery.TableDestination
24 | import org.slf4j.LoggerFactory
25 |
26 | object WriteDynamicTable {
27 |
28 | private val log = LoggerFactory.getLogger(this.getClass)
29 |
30 | // Define data class to write to BQ, schema is inferred
31 | @BigQueryType.toTable
32 | case class Person(
33 | name: String,
34 | age: Int,
35 | country: String,
36 | group: Int
37 | )
38 |
39 | def main(cmdlineArgs: Array[String]): Unit = {
40 | val (scontext: ScioContext, opts: Args) = ContextAndArgs(cmdlineArgs)
41 | implicit val sc: ScioContext = scontext
42 |
43 | // pipeline option parameters
44 | val tablePrefix: String = opts("tablePrefix")
45 |
46 | val elements = Seq(
47 | ("Carlos", 23, "Spain", 1),
48 | ("Maria", 27, "UK", 2),
49 | ("Ruben", 15, "Japan", 1),
50 | ("Juana", 38, "Ghana", 1),
51 | ("Kim", 77, "Brazil", 2)
52 | )
53 |
54 |
55 | val create: SCollection[(String, Int, String, Int)] = sc.parallelize(elements)
56 |
57 | create
58 | .map { row: (String, Int, String, Int) => Person(row._1, row._2, row._3, row._4) }
59 | .map[TableRow](Person.toTableRow)
60 | .saveAsBigQuery(Person.schema, WRITE_EMPTY, CREATE_IF_NEEDED) { r =>
61 | val group = r.getValue.get("group").toString
62 | new TableDestination(s"${tablePrefix}_$group", s"table for group $group")
63 | }
64 |
65 | sc.run()
66 | }
67 |
68 |
69 | }
70 |
--------------------------------------------------------------------------------
/Scala/src/main/scala/bigquery/WriteStreamingInserts.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2022 Google LLC
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * https://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package bigquery
18 |
19 | import com.spotify.scio.bigquery._
20 | import com.spotify.scio.pubsub._
21 | import com.spotify.scio.{Args, ContextAndArgs, ScioContext}
22 | import io.circe._
23 | import io.circe.generic.semiauto._
24 | import io.circe.parser.decode
25 | import org.slf4j.LoggerFactory
26 |
27 |
28 | object WriteStreamingInserts {
29 |
30 | private val log = LoggerFactory.getLogger(this.getClass)
31 |
32 | // Schema for parsing
33 | @BigQueryType.toTable
34 | case class Taxi(
35 | ride_id: String,
36 | timestamp: String,
37 | meter_reading: Double,
38 | ride_status: String,
39 | passenger_count: Int
40 | )
41 |
42 | implicit val taxiRideDecoder: Decoder[Taxi] = deriveDecoder[Taxi]
43 |
44 | def main(cmdlineArgs: Array[String]): Unit = {
45 | val (scontext: ScioContext, opts: Args) = ContextAndArgs(cmdlineArgs)
46 | implicit val sc: ScioContext = scontext
47 |
48 | // pipeline option parameters
49 | val pubsubTopic: String = opts.getOrElse("topic", "projects/pubsub-public-data/topics/taxirides-realtime")
50 | val table: String = opts("table")
51 |
52 | val pubsubRead = sc.read(PubsubIO.string(pubsubTopic))(PubsubIO.ReadParam(PubsubIO.Topic))
53 |
54 | pubsubRead
55 | .withName("Parse JSON")
56 | .map { json: String =>
57 | val parsed: Either[Error, Taxi] = decode[Taxi](json)
58 | parsed
59 | }
60 | .collect { case Right(p) => p }
61 | .saveAsBigQueryTable(Table.Spec(table), WRITE_APPEND, CREATE_IF_NEEDED)
62 |
63 |
64 | sc.run()
65 | }
66 |
67 |
68 | }
69 |
--------------------------------------------------------------------------------
/Scala/src/main/scala/extra/ClassicTemplate.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2022 Google LLC
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * https://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package extra
18 |
19 | import com.spotify.scio.ScioContext
20 | import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions
21 | import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO
22 | import org.apache.beam.sdk.options.{Default, Description, PipelineOptionsFactory, ValueProvider}
23 | import org.slf4j.LoggerFactory
24 |
25 | object ClassicTemplate {
26 | /*
27 | This example shows how to use Classic Templates. Note that Flex Templates
28 | are the preferred method.
29 | */
30 |
31 | private val log = LoggerFactory.getLogger(this.getClass)
32 |
33 | trait TemplateOptions extends DataflowPipelineOptions {
34 | @Description("BigQuery Table")
35 | @Default.String("bigquery-public-data:census_bureau_usa.population_by_zip_2010")
36 | def getTable: ValueProvider[String]
37 |
38 | def setTable(value: ValueProvider[String]): Unit
39 | }
40 |
41 | def main(cmdlineArgs: Array[String]): Unit = {
42 |
43 | PipelineOptionsFactory.register(classOf[TemplateOptions])
44 | val options = PipelineOptionsFactory.fromArgs(cmdlineArgs: _*).withValidation.as(classOf[TemplateOptions])
45 | options.setStreaming(false)
46 |
47 | val sc = ScioContext(options)
48 |
49 | // Since we are using ValueProviders, we need custom IO
50 | val customIO = BigQueryIO.readTableRows()
51 | // notice is options.getTable and not options.getTable() as in Java
52 | .from(options.getTable)
53 | .withoutValidation()
54 | .withTemplateCompatibility()
55 |
56 | val read = sc.customInput("BigQuery read", customIO)
57 |
58 | read
59 | .count
60 | .map { c: Long => log.info(s"Total count $c") }
61 |
62 | sc.run()
63 | }
64 |
65 |
66 | }
67 |
--------------------------------------------------------------------------------
/Scala/src/main/scala/gcs/AvroInOut.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2022 Google LLC
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * https://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package gcs
18 |
19 | import com.spotify.scio.avro._
20 | import com.spotify.scio.coders._
21 | import com.spotify.scio.values.SCollection
22 | import com.spotify.scio.{Args, ContextAndArgs, ScioContext}
23 | import org.apache.avro.generic.{GenericData, GenericRecord}
24 | import org.slf4j.LoggerFactory
25 |
26 |
27 | object AvroInOut {
28 |
29 | private val log = LoggerFactory.getLogger(this.getClass)
30 |
31 | @AvroType.fromSchema(
32 | """{
33 | | "type":"record",
34 | | "name":"State",
35 | | "namespace":"scio.cookbook",
36 | | "doc":"Record for a US state",
37 | | "fields":[
38 | | {"name":"name","type":"string"},
39 | | {"name":"post_abbr","type":"string"}]}
40 | """.stripMargin)
41 | class State
42 |
43 | def main(cmdlineArgs: Array[String]): Unit = {
44 | val (scontext: ScioContext, opts: Args) = ContextAndArgs(cmdlineArgs)
45 | implicit val sc: ScioContext = scontext
46 |
47 | // pipeline option parameters
48 | val input: String = opts.getOrElse("input", "gs://cloud-samples-data/bigquery/us-states/*.avro")
49 | val output: String = opts("output")
50 |
51 | // Coder helps performance with GenericRecords
52 | implicit def genericCoder = Coder.avroGenericRecordCoder(State.schema)
53 |
54 | val read: SCollection[GenericRecord] = sc.avroFile(input, State.schema)
55 |
56 | read
57 | .map[GenericRecord] { record: GenericRecord =>
58 | val name = record.get("name").toString
59 | val postAbbr = record.get("post_abbr").toString
60 | log.info(s"The abbreviation of $name is $postAbbr")
61 | val newRecord = new GenericData.Record(State.schema)
62 | newRecord.put("name", name)
63 | newRecord.put("post_abbr", postAbbr.toLowerCase())
64 | newRecord
65 | }
66 | .saveAsAvroFile(output, schema = State.schema)
67 |
68 | sc.run()
69 | }
70 | }
71 |
--------------------------------------------------------------------------------
/Scala/src/main/scala/gcs/ReadFiles.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2022 Google LLC
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * https://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package gcs
18 |
19 | import com.spotify.scio.values.SCollection
20 | import com.spotify.scio.{Args, ContextAndArgs, ScioContext}
21 | import org.slf4j.LoggerFactory
22 |
23 |
24 | object ReadFiles {
25 |
26 | private val log = LoggerFactory.getLogger(this.getClass)
27 |
28 | def main(cmdlineArgs: Array[String]): Unit = {
29 | val (scontext: ScioContext, opts: Args) = ContextAndArgs(cmdlineArgs)
30 | implicit val sc: ScioContext = scontext
31 |
32 | val elements = Seq(
33 | "gs://apache-beam-samples/shakespeare/kinglear.txt",
34 | "gs://apache-beam-samples/shakespeare/macbeth.txt",
35 | "gs://apache-beam-samples/shakespeare/a*.txt"
36 | )
37 |
38 |
39 | val files: SCollection[String] = sc.parallelize(elements)
40 |
41 | files
42 | .readFiles
43 | .map(x => log.info(x))
44 |
45 |
46 | sc.run()
47 | }
48 | }
49 |
--------------------------------------------------------------------------------
/Scala/src/main/scala/gcs/ReadText.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2022 Google LLC
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * https://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package gcs
18 |
19 | import com.spotify.scio.values.SCollection
20 | import com.spotify.scio.{Args, ContextAndArgs, ScioContext}
21 | import org.slf4j.LoggerFactory
22 |
23 |
24 | object ReadText {
25 |
26 | private val log = LoggerFactory.getLogger(this.getClass)
27 |
28 | def main(cmdlineArgs: Array[String]): Unit = {
29 | val (scontext: ScioContext, opts: Args) = ContextAndArgs(cmdlineArgs)
30 | implicit val sc: ScioContext = scontext
31 |
32 | // pipeline option parameters
33 | val input: String = opts.getOrElse("input", "gs://apache-beam-samples/shakespeare/kinglear.txt")
34 |
35 | val read: SCollection[String] = sc.textFile(input)
36 |
37 | read.map(log.info(_))
38 |
39 |
40 | sc.run()
41 | }
42 | }
43 |
--------------------------------------------------------------------------------
/Scala/src/main/scala/gcs/WriteDynamic.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2022 Google LLC
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * https://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package gcs
18 |
19 | import com.spotify.scio.io.dynamic._
20 | import com.spotify.scio.values.SCollection
21 | import com.spotify.scio.{Args, ContextAndArgs, ScioContext}
22 | import org.slf4j.LoggerFactory
23 |
24 |
25 | object WriteDynamic {
26 |
27 | private val log = LoggerFactory.getLogger(this.getClass)
28 |
29 | def main(cmdlineArgs: Array[String]): Unit = {
30 | val (scontext: ScioContext, opts: Args) = ContextAndArgs(cmdlineArgs)
31 | implicit val sc: ScioContext = scontext
32 |
33 | // pipeline option parameters
34 | val output: String = opts("output")
35 |
36 | val create: SCollection[Int] = sc.parallelize(2 to 1000)
37 |
38 | create
39 | .map {
40 | _.toString
41 | }
42 | .saveAsDynamicTextFile(output)(isPrime(_))
43 |
44 |
45 | def isPrime(n: String): String = {
46 | val number = n.toInt
47 | val squareRoot = math.sqrt(number).toInt
48 | (2 until squareRoot + 1) forall (i => number % i != 0) match {
49 | case true => "prime"
50 | case false => "not-prime"
51 | }
52 | }
53 |
54 | sc.run()
55 | }
56 | }
57 |
--------------------------------------------------------------------------------
/Scala/src/main/scala/gcs/WriteText.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2022 Google LLC
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * https://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package gcs
18 |
19 | import com.spotify.scio.values.SCollection
20 | import com.spotify.scio.{Args, ContextAndArgs, ScioContext}
21 | import org.slf4j.LoggerFactory
22 |
23 |
24 | object WriteText {
25 |
26 | private val log = LoggerFactory.getLogger(this.getClass)
27 |
28 | def main(cmdlineArgs: Array[String]): Unit = {
29 | val (scontext: ScioContext, opts: Args) = ContextAndArgs(cmdlineArgs)
30 | implicit val sc: ScioContext = scontext
31 |
32 | // pipeline option parameters
33 | val output: String = opts("output")
34 |
35 | val elements = Seq(
36 | "En un lugar de la Mancha",
37 | "de cuyo nombre no quiero acordarme",
38 | "no ha mucho tiempo que vivia un hidalgo",
39 | "de los de lanza en astillero, adarga antigua, rocin flaco y galgo corredor"
40 | )
41 |
42 |
43 | val create: SCollection[String] = sc.parallelize(elements)
44 |
45 | create
46 | .saveAsTextFile(output, numShards = 0, suffix = ".txt")
47 |
48 |
49 | sc.run()
50 | }
51 | }
52 |
--------------------------------------------------------------------------------
/Scala/src/main/scala/gcs/WriteTextStreaming.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2022 Google LLC
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * https://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package gcs
18 |
19 | import com.spotify.scio.pubsub._
20 | import com.spotify.scio.{Args, ContextAndArgs, ScioContext}
21 | import io.circe._
22 | import io.circe.generic.semiauto._
23 | import io.circe.parser.decode
24 | import org.apache.beam.sdk.io.TextIO
25 | import org.joda.time.Duration
26 | import org.slf4j.LoggerFactory
27 |
28 |
29 | object WriteTextStreaming {
30 |
31 | private val log = LoggerFactory.getLogger(this.getClass)
32 |
33 | case class Taxi(
34 | ride_id: String,
35 | meter_reading: Double,
36 | ride_status: String
37 | )
38 |
39 | implicit val taxiRideDecoder: Decoder[Taxi] = deriveDecoder[Taxi]
40 |
41 | def main(cmdlineArgs: Array[String]): Unit = {
42 | val (scontext: ScioContext, opts: Args) = ContextAndArgs(cmdlineArgs)
43 | implicit val sc: ScioContext = scontext
44 |
45 | // pipeline option parameters
46 | val output: String = opts("output")
47 | val pubsubTopic: String = opts.getOrElse("topic", "projects/pubsub-public-data/topics/taxirides-realtime")
48 |
49 | val pubsubRead = sc.read(PubsubIO.string(pubsubTopic))(PubsubIO.ReadParam(PubsubIO.Topic))
50 |
51 | pubsubRead
52 | .withName("Parse JSON")
53 | .map { json: String =>
54 | val parsed: Either[Error, Taxi] = decode[Taxi](json)
55 | parsed
56 | }
57 | .collect { case Right(p) => p }
58 | .filter { t: Taxi => t.ride_status.equals("dropoff") }
59 | .map { t: Taxi => s"Taxi id ${t.ride_id} costed ${t.meter_reading}" }
60 | .withFixedWindows(Duration.standardMinutes(10))
61 | // We need a custom IO output
62 | .saveAsCustomOutput(
63 | "Write Streaming File(s)",
64 | TextIO.write()
65 | .withWindowedWrites()
66 | .withNumShards(2)
67 | .to(output)
68 | )
69 |
70 |
71 | sc.run()
72 | }
73 | }
74 |
--------------------------------------------------------------------------------
/Scala/src/main/scala/minimal/minimal.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2022 Google LLC
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * https://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package minimal
18 |
19 | import com.spotify.scio.{Args, ContextAndArgs, ScioContext}
20 | import org.slf4j.LoggerFactory
21 |
22 |
23 | object minimal {
24 |
25 | private val log = LoggerFactory.getLogger(this.getClass)
26 |
27 | def main(cmdlineArgs: Array[String]): Unit = {
28 | val (scontext: ScioContext, opts: Args) = ContextAndArgs(cmdlineArgs)
29 | implicit val sc: ScioContext = scontext
30 |
31 | // add steps here
32 | sc.run()
33 | }
34 | }
35 |
--------------------------------------------------------------------------------
/Scala/src/main/scala/pubsub/ReadTopic.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2022 Google LLC
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * https://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package pubsub
18 |
19 | import com.spotify.scio.pubsub._
20 | import com.spotify.scio.{Args, ContextAndArgs, ScioContext}
21 | import io.circe._
22 | import io.circe.generic.semiauto._
23 | import io.circe.parser.decode
24 | import org.slf4j.LoggerFactory
25 |
26 |
27 | object ReadTopic {
28 |
29 | private val log = LoggerFactory.getLogger(this.getClass)
30 |
31 | // Schema for parsing
32 | case class Taxi(
33 | ride_id: String,
34 | timestamp: String,
35 | meter_reading: Double,
36 | ride_status: String,
37 | passenger_count: Int
38 | )
39 |
40 | implicit val taxiRideDecoder: Decoder[Taxi] = deriveDecoder[Taxi]
41 |
42 | def main(cmdlineArgs: Array[String]): Unit = {
43 | val (scontext: ScioContext, opts: Args) = ContextAndArgs(cmdlineArgs)
44 | implicit val sc: ScioContext = scontext
45 |
46 | // pipeline option parameters
47 | val pubsubTopic: String = opts.getOrElse("topic", "projects/pubsub-public-data/topics/taxirides-realtime")
48 |
49 | val pubsubRead = sc.read(PubsubIO.string(pubsubTopic))(PubsubIO.ReadParam(PubsubIO.Topic))
50 |
51 | pubsubRead
52 | .map { json: String =>
53 | val parsed: Either[Error, Taxi] = decode[Taxi](json)
54 | parsed
55 | }
56 | .collect { case Right(p) => p }
57 | .filter { t => t.ride_status == "dropoff" }
58 | .map { ride =>
59 | log.info(s"${ride.ride_id} costed ${ride.meter_reading} for ${ride.passenger_count} passengers")
60 | }
61 |
62 |
63 | sc.run()
64 | }
65 |
66 |
67 | }
68 |
--------------------------------------------------------------------------------
/Scala/src/main/scala/pubsub/ReadWithAttributes.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2022 Google LLC
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * https://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package pubsub
18 |
19 | import com.spotify.scio.pubsub._
20 | import com.spotify.scio.{Args, ContextAndArgs, ScioContext}
21 | import org.slf4j.LoggerFactory
22 |
23 |
24 | object ReadWithAttributes {
25 |
26 | private val log = LoggerFactory.getLogger(this.getClass)
27 |
28 | def main(cmdlineArgs: Array[String]): Unit = {
29 | val (scontext: ScioContext, opts: Args) = ContextAndArgs(cmdlineArgs)
30 | implicit val sc: ScioContext = scontext
31 |
32 | // pipeline option parameters
33 | // messages optionally have attributes "key" and "country" (Strings)
34 | val pubsubTopic: String = opts("topic")
35 |
36 | val pubsubRead = sc.read(PubsubIO.withAttributes[String](pubsubTopic))(PubsubIO.ReadParam(PubsubIO.Topic))
37 |
38 | pubsubRead
39 | .map { message =>
40 | val payload = new String(message._1)
41 | val attributes = message._2
42 | val age = attributes.get("age")
43 | val country = attributes.get("country")
44 |
45 | val string = (age, country) match {
46 | case (Some(a), Some(c)) => s" has age $a and is from $c."
47 | case (Some(a), _) => s" has age $a."
48 | case (_, Some(c)) => s" is from $c."
49 | case (_, _) => "."
50 | }
51 |
52 | s"$payload$string"
53 | }
54 | .map { p: String => log.info(p) }
55 |
56 |
57 | sc.run()
58 | }
59 |
60 |
61 | }
62 |
--------------------------------------------------------------------------------
/Scala/src/main/scala/pubsub/WriteTopic.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2022 Google LLC
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * https://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package pubsub
18 |
19 | import com.spotify.scio.pubsub._
20 | import com.spotify.scio.{Args, ContextAndArgs, ScioContext}
21 | import io.circe._
22 | import io.circe.generic.semiauto._
23 | import org.slf4j.LoggerFactory
24 |
25 |
26 | object WriteTopic {
27 |
28 | private val log = LoggerFactory.getLogger(this.getClass)
29 |
30 | def main(cmdlineArgs: Array[String]): Unit = {
31 | val (scontext: ScioContext, opts: Args) = ContextAndArgs(cmdlineArgs)
32 | implicit val sc: ScioContext = scontext
33 |
34 | // pipeline option parameter
35 | val pubsubOutputTopic: String = opts("outputTopic")
36 |
37 | val topic = "projects/pubsub-public-data/topics/taxirides-realtime"
38 |
39 | // Reading from a topic creates a subscription to it automatically.
40 | val pubsubRead = sc.read(PubsubIO.string(topic))(PubsubIO.ReadParam(PubsubIO.Topic))
41 |
42 | pubsubRead
43 | // Since we read strings, we can publish directly as strings
44 | .write(PubsubIO.string(pubsubOutputTopic))(PubsubIO.WriteParam())
45 |
46 |
47 | sc.run()
48 | }
49 |
50 |
51 | }
52 |
--------------------------------------------------------------------------------
/Scala/src/main/scala/pubsub/WriteWithAttributes.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2022 Google LLC
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * https://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package pubsub
18 |
19 | import com.spotify.scio.pubsub._
20 | import com.spotify.scio.values.SCollection
21 | import com.spotify.scio.{Args, ContextAndArgs, ScioContext}
22 | import org.slf4j.LoggerFactory
23 |
24 | object WriteWithAttributes {
25 |
26 | private val log = LoggerFactory.getLogger(this.getClass)
27 |
28 | def main(cmdlineArgs: Array[String]): Unit = {
29 | val (scontext: ScioContext, opts: Args) = ContextAndArgs(cmdlineArgs)
30 | implicit val sc: ScioContext = scontext
31 |
32 | // pipeline option parameter
33 | val pubsubOutputTopic: String = opts("outputTopic")
34 |
35 | val elements = Seq(
36 | ("Carlos", 27, "Spain"),
37 | ("Maria", 27, "UK"),
38 | ("Ruben", 15, "Japan"),
39 | ("Juana", 38, "Ghana"),
40 | ("Kim", 77, "Brazil")
41 | )
42 |
43 | val create: SCollection[(String, Int, String)] = sc.parallelize(elements)
44 |
45 | val pubsubMessages = create
46 | .map { element: (String, Int, String) =>
47 | // Map has to be [String, String]
48 | val attributesMap = Map(
49 | "age" -> element._2.toString,
50 | "country" -> element._3
51 | )
52 | (element._1, attributesMap)
53 | }
54 | .write(PubsubIO.withAttributes[String](pubsubOutputTopic))(PubsubIO.WriteParam())
55 |
56 | sc.run()
57 | }
58 |
59 |
60 | }
61 |
--------------------------------------------------------------------------------
/Scala/src/main/scala/windows/SessionWindows.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2022 Google LLC
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * https://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package windows
18 |
19 | import com.spotify.scio.pubsub._
20 | import com.spotify.scio.{Args, ContextAndArgs, ScioContext}
21 | import io.circe._
22 | import io.circe.generic.semiauto._
23 | import io.circe.parser.decode
24 | import org.joda.time.Duration
25 | import org.slf4j.LoggerFactory
26 |
27 |
28 | object SessionWindows {
29 |
30 | private val log = LoggerFactory.getLogger(this.getClass)
31 |
32 | case class Taxi(
33 | ride_id: String,
34 | ride_status: String,
35 | meter_increment: Double
36 | )
37 |
38 | implicit val taxiRideDecoder: Decoder[Taxi] = deriveDecoder[Taxi]
39 |
40 | def main(cmdlineArgs: Array[String]): Unit = {
41 | val (scontext: ScioContext, opts: Args) = ContextAndArgs(cmdlineArgs)
42 | implicit val sc: ScioContext = scontext
43 |
44 | // pipeline option parameters
45 | val gap = opts.getOrElse("duration", "10")
46 | val pubsubTopic: String = opts.getOrElse("topic", "projects/pubsub-public-data/topics/taxirides-realtime")
47 |
48 | val pubsubRead = sc.read(PubsubIO.string(pubsubTopic))(PubsubIO.ReadParam(PubsubIO.Topic))
49 |
50 | pubsubRead
51 | .withName("Parse JSON")
52 | .map { json: String =>
53 | val parsed: Either[Error, Taxi] = decode[Taxi](json)
54 | parsed
55 | }
56 | .collect { case Right(p) => p }
57 | .filter { t: Taxi => t.ride_status.equals("enroute") }
58 | .map { t: Taxi => (t.ride_id, t.meter_increment) }
59 | .withSessionWindows(Duration.standardMinutes(gap.toInt))
60 | .sumByKey
61 | .map { s => s"Total cost of ride \'${s._1}\': ${s._2}" }
62 | .withName("log")
63 | .map {
64 | log.info(_)
65 | }
66 |
67 |
68 | sc.run()
69 | }
70 | }
71 |
--------------------------------------------------------------------------------
/Scala/src/main/scala/windows/SlidingWindows.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2022 Google LLC
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * https://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package windows
18 |
19 | import com.spotify.scio.pubsub._
20 | import com.spotify.scio.{Args, ContextAndArgs, ScioContext}
21 | import io.circe._
22 | import io.circe.generic.semiauto._
23 | import io.circe.parser.decode
24 | import org.joda.time.Duration
25 | import org.slf4j.LoggerFactory
26 |
27 |
28 | object SlidingWindows {
29 |
30 | private val log = LoggerFactory.getLogger(this.getClass)
31 |
32 | case class Taxi(
33 | ride_status: String
34 | )
35 |
36 | implicit val taxiRideDecoder: Decoder[Taxi] = deriveDecoder[Taxi]
37 |
38 | def main(cmdlineArgs: Array[String]): Unit = {
39 | val (scontext: ScioContext, opts: Args) = ContextAndArgs(cmdlineArgs)
40 | implicit val sc: ScioContext = scontext
41 |
42 | // pipeline option parameters
43 | val duration = opts.getOrElse("duration", "10")
44 | val frequency = opts.getOrElse("frequency", "2")
45 | val pubsubTopic: String = opts.getOrElse("topic", "projects/pubsub-public-data/topics/taxirides-realtime")
46 |
47 | val pubsubRead = sc.read(PubsubIO.string(pubsubTopic))(PubsubIO.ReadParam(PubsubIO.Topic))
48 |
49 | pubsubRead
50 | .withName("Parse JSON")
51 | .map { json: String =>
52 | val parsed: Either[Error, Taxi] = decode[Taxi](json)
53 | parsed
54 | }
55 | .collect { case Right(p) => p }
56 | .withSlidingWindows(Duration.standardMinutes(duration.toInt), Duration.standardMinutes(frequency.toInt))
57 | .map { t: Taxi => (t.ride_status, 1) }
58 | .countByKey
59 | .map { s => s"Total messages for key ${s._1}: ${s._2}" }
60 | .withName("log")
61 | .map {
62 | log.info(_)
63 | }
64 |
65 |
66 | sc.run()
67 | }
68 | }
69 |
--------------------------------------------------------------------------------