Beam Testing Documentation

Overview

Apache Beam is a rapidly-maturing software project with a strong commitment to testing. Consequently, it has many testing-related needs. It requires precommit tests to ensure code going into the repository meets a certain quality bar and it requires ongoing postcommit tests to make sure that more subtle changes which escape precommit are nonetheless caught. This document outlines how to write tests, which tests are appropriate where, and when tests are run, with some additional information about the testing systems at the bottom.

If you’re writing tests, take a look at the testing matrix first, find what you want to test, then look into the “Scenarios” and “Types” sections below for more details on those testing types.

Testing Matrix

Java SDK

Component to Test Test Scenario Tool to Use Link to Example Type Runs In
DoFn Correctness on one/few bundles DoFnTester BigQueryIOTest Unit Precommit, Postcommit
BoundedSource Correctly Reads Input SourceTestUtils.readFromSource TextIOTest Unit Precommit, Postcommit
Correct Initial Splitting SourceTestUtils.assertSourcesEqualReferenceSource BigtableTest Unit Precommit, Postcommit
Correct Dynamic Splitting SourceTestUtils. assertSplitAtFractionExhaustive TextIOTest Unit Precommit, Postcommit
Transform Correctness @NeedsRunner Test ParDoTest @NeedsRunner
Example Pipeline Verify Behavior on Each Runner E2E Test WordCountIT E2E Postcommit (Except WordCountIT)
Source/Sink with external resource External Resource Faked Unit / @NeedsRunner Test FakeBigtableService in BigtableTest Unit / @NeedsRunner Precommit / Postcommit
Real Interactions With External Resource E2E Test BigtableReadIT E2E Postcommit
Runner Correctness E2E Test, @RunnableonService WordCountIT, ParDoTest E2E, @RunnableonService Postcommit
Coders Encoding/decoding elements CoderProperties NullableCoderTest Unit Precommit / Postcommit
Serialization/deserialization of Coder CoderProperties NullableCoderTest Unit Precommit / Postcommit
Sizing of elements CoderProperties NullableCoderTest Unit Precommit / Postcommit
Deterministic CoderProperties NullableCoderTest Unit Precommit / Postcommit
Structural value equality CoderProperties NullableCoderTest Unit Precommit / Postcommit

Python SDK

The Python SDK is currently under development on a feature branch. We have initial postcommit tests by a Jenkins build; precommit testing and a full testing matrix will be coming soon.

Testing Scenarios

With the tools at our disposal, we have a good set of utilities which we can use to verify Beam correctness. To ensure an ongoing high quality of code, we use precommit and postcommit testing.

Precommit

For precommit testing, Beam uses Jenkins, Travis, and a code coverage tool called Coveralls, hooked up to Github, to ensure that pull requests meet a certain quality bar. These precommits verify correctness via two of the below testing tools: unit tests (with coverage monitored by Coveralls) and E2E tests. We run the full slate of unit tests in precommit, ensuring correctness at a basic level, and then run the WordCount E2E test in both batch and streaming (coming soon!) against each supported SDK / runner combination as a smoke test, to verify that a basic level of functionality exists. We think that this hits the appropriate tradeoff between a desire for short (ideally <30m) precommit times and a desire to verify that pull requests going into Beam function in the way in which they are intended.

Precommit tests are kicked off when a user makes a Pull Request against the apache/beam repository and the Travis, Jenkins, and Coveralls statuses are displayed at the bottom of the pull request page. Clicking on “Details” will open the status page in the selected tool; there, test status and output can be viewed.

Postcommit

Running in postcommit removes as stringent of a time constraint, which gives us the ability to do some more comprehensive testing. In postcommit we have a test suite running the RunnableOnService tests against each supported runner, and another for running the full set of E2E tests against each runner. Currently-supported runners are Dataflow, Flink, Spark, and Gearpump, with others soon to follow. Work is ongoing to enable Flink, Spark, and Gearpump in the E2E framework, with full support targeted for end of August 2016. Postcommit tests run periodically, with timing defined in their Jenkins configurations.

Adding new postcommit E2E tests is generally as easy as adding a *IT.java file to the repository - Failsafe will notice it and run it - but if you want to do more interesting things, take a look at WordCountIT.java.

Postcommit test results can be found in Jenkins.

Testing Types

Unit

Unit tests are, in Beam as everywhere else, the first line of defense in ensuring software correctness. As all of the contributors to Beam understand the importance of testing, Beam has a robust set of unit tests, as well as testing coverage measurement tools, which protect the codebase from simple to moderate breakages. Beam Java unit tests are written in JUnit.

RunnableOnService (Working Title)

RunnableOnService tests contain components of both component and end-to-end tests. They fulfill the typical purpose of a component test - they are meant to test a well-scoped piece of Beam functionality or the interactions between two such pieces and can be run in a component-test-type fashion against the DirectRunner. Additionally, they are built with the ability to run in an end-to-end fashion against a runner, allowing them to verify not only core Beam functionality, but runner functionality as well. They are more lightweight than a traditional end-to-end test and, because of their well-scoped nature, provide good signal as to what exactly is working or broken against a particular runner.

The name “RunnableOnService” is an artifact of when Beam was still the Google Cloud Dataflow SDK and will be changing to something more indicative of its use in the coming months.

E2E

End-to-End tests are meant to verify at the very highest level that the Beam codebase is working as intended. Because they are implemented as a thin wrapper around existing pipelines, they can be used to prove that the core Beam functionality is available. They will be used to verify runner correctness, but they can also be used for IO connectors and other core functionality.

Testing Systems

E2E Testing Framework

The Beam end-to-end testing framework is a framework designed in a runner-agnostic fashion to exercise the entire lifecycle of a Beam pipeline. We run a pipeline as a user would and allow it to run to completion in the same way, verifying after completion that it behaved how we expected. Using pipelines from the Beam examples, or custom-built pipelines, the framework will provide hooks during several pipeline lifecycle events, e.g., pipeline creation, pipeline success, and pipeline failure, to allow verification of pipeline state.

The E2E testing framework is currently built to hook into the Maven Failsafe Integration Test plugin, which means it is tightly integrated with the overall build process. Once it is determined how Python and other future languages will integrate into the overall build/test system (via Maven or otherwise) we will adjust this. The framework provides a wrapper around actual Beam pipelines, enabling those pipelines to be run in an environment which facilitates verification of pipeline results and details.

Verifiers include:

The E2E framework will support running on various different configurations of environments. We currently provide the ability to run against the DirectRunner, against a local Spark instance, a local Flink instance, and against the Google Cloud Dataflow service.

RunnableOnService Tests

RunnableOnService tests are tests built to use the Beam TestPipeline class, which enables test authors to write simple functionality verification. They are meant to use some of the built-in utilities of the SDK, namely PAssert, to verify that the simple pipelines they run end in the correct state.