├── .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 | --------------------------------------------------------------------------------