Skip to content

Commit

Permalink
This closes #465
Browse files Browse the repository at this point in the history
  • Loading branch information
Mergebot committed Jul 16, 2018
2 parents b25b152 + 3617080 commit a9b0c6b
Showing 1 changed file with 203 additions and 20 deletions.
223 changes: 203 additions & 20 deletions src/documentation/io/testing.md
Expand Up @@ -147,9 +147,9 @@ However, when working locally, there is no requirement to use Kubernetes. All of
### Running integration tests {#running-integration-tests}

The high level steps for running an integration test are:
1. Set up the data store corresponding to the test being run
1. Run the test, passing it connection info from the just created data store
1. Clean up the data store
1. Set up the data store corresponding to the test being run.
1. Run the test, passing it connection info from the just created data store.
1. Clean up the data store.

Since setting up data stores and running the tests involves a number of steps, and we wish to time these tests when running performance benchmarks, we use PerfKit Benchmarker to manage the process end to end. With a single command, you can go from an empty Kubernetes cluster to a running integration test.

Expand All @@ -160,21 +160,28 @@ However, **PerfKit Benchmarker is not required for running integration tests**.

Prerequisites:
1. [Install PerfKit Benchmarker](https://github.com/GoogleCloudPlatform/PerfKitBenchmarker)
1. Have a running Kubernetes cluster you can connect to locally using kubectl
1. Have a running Kubernetes cluster you can connect to locally using kubectl. We recommend using Google Kubernetes Engine - it's proven working for all the use cases we tested.

You won’t need to invoke PerfKit Benchmarker directly. Run `./gradlew performanceTest` in project's root directory, passing appropriate kubernetes scripts depending on the network you're using (local network or remote one).
You won’t need to invoke PerfKit Benchmarker directly. Run `./gradlew performanceTest` task in project's root directory, passing kubernetes scripts of your choice (located in .test_infra/kubernetes directory). It will setup PerfKitBenchmarker for you.

Example run with the direct runner:
Example run with the [Direct](https://beam.apache.org/documentation/runners/direct/) runner:
```
./gradlew performanceTest -DpkbLocation="/Users/me/PerfKitBenchmarker/pkb.py" -DintegrationTestPipelineOptions='["--numberOfRecords=1000"]' -DitModule=sdks/java/io/jdbc/ -DintegrationTest=org.apache.beam.sdk.io.jdbc.JdbcIOIT -DkubernetesScripts="/Users/me/beam/.test-infra/kubernetes/postgres/postgres-service-for-local-dev.yml" -DbeamITOptions="/Users/me/beam/.test-infra/kubernetes/postgres/pkb-config-local.yml" -DintegrationTestRunner=direct
```


Example run with the Cloud Dataflow runner:
Example run with the [Google Cloud Dataflow](https://beam.apache.org/documentation/runners/dataflow/) runner:
```
/gradlew performanceTest -DpkbLocation="/Users/me/PerfKitBenchmarker/pkb.py" -DintegrationTestPipelineOptions='["--numberOfRecords=1000", "--project=GOOGLE_CLOUD_PROJECT", "--tempRoot=GOOGLE_STORAGE_BUCKET"]' -DitModule=sdks/java/io/jdbc/ -DintegrationTest=org.apache.beam.sdk.io.jdbc.JdbcIOIT -DkubernetesScripts="/Users/me/beam/.test-infra/kubernetes/postgres/postgres-service-for-local-dev.yml" -DbeamITOptions="/Users/me/beam/.test-infra/kubernetes/postgres/pkb-config-local.yml" -DintegrationTestRunner=dataflow
./gradlew performanceTest -DpkbLocation="/Users/me/PerfKitBenchmarker/pkb.py" -DintegrationTestPipelineOptions='["--numberOfRecords=1000", "--project=GOOGLE_CLOUD_PROJECT", "--tempRoot=GOOGLE_STORAGE_BUCKET"]' -DitModule=sdks/java/io/jdbc/ -DintegrationTest=org.apache.beam.sdk.io.jdbc.JdbcIOIT -DkubernetesScripts="/Users/me/beam/.test-infra/kubernetes/postgres/postgres-service-for-local-dev.yml" -DbeamITOptions="/Users/me/beam/.test-infra/kubernetes/postgres/pkb-config-local.yml" -DintegrationTestRunner=dataflow
```

Example run with the HDFS filesystem and Cloud Dataflow runner:

```
./gradlew performanceTest -DpkbLocation="/Users/me/PerfKitBenchmarker/pkb.py" -DintegrationTestPipelineOptions='["--numberOfRecords=100000", "--project=GOOGLE_CLOUD_PROJECT", "--tempRoot=GOOGLE_STORAGE_BUCKET"]' -DitModule=sdks/java/io/file-based-io-tests/ -DintegrationTest=org.apache.beam.sdk.io.text.TextIOIT -DkubernetesScripts=".test-infra/kubernetes/hadoop/LargeITCluster/hdfs-multi-datanode-cluster.yml,.test-infra/kubernetes/hadoop/LargeITCluster/hdfs-multi-datanode-cluster-for-local-dev.yml" -DbeamITOptions=".test-infra/kubernetes/hadoop/LargeITCluster/pkb-config.yml" -DintegrationTestRunner=dataflow -DbeamExtraProperties='[filesystem=hdfs]'
```

NOTE: When using Direct runner along with HDFS cluster, please set `export HADOOP_USER_NAME=root` before runnning `performanceTest` task.

Parameter descriptions:

Expand All @@ -200,7 +207,7 @@ Parameter descriptions:
<tr>
<td>-DintegrationTestPipelineOptions
</td>
<td>Passes pipeline options directly to the test being run.
<td>Passes pipeline options directly to the test being run. Note that some pipeline options may be runner specific (like "--project" or "--tempRoot").
</td>
</tr>
<tr>
Expand All @@ -212,7 +219,7 @@ Parameter descriptions:
<tr>
<td>-DintegrationTest
</td>
<td>Specifies the test to be run.
<td>Specifies the test to be run (fully qualified reference to class/test method).
</td>
</tr>
<tr>
Expand All @@ -233,31 +240,209 @@ Parameter descriptions:
<td>Runner to be used for running the test. Currently possible options are: direct, dataflow.
</td>
</tr>
<tr>
<td>-DbeamExtraProperties
</td>
<td>Any other "extra properties" to be passed to Gradle, eg. "'[filesystem=hdfs]'".
</td>
</tr>
</tbody>
</table>



#### Without PerfKit Benchmarker {#without-perfkit-benchmarker}

If you're using Kubernetes, make sure you can connect to your cluster locally using kubectl. Otherwise, skip to step 3 below.
If you're using Kubernetes scripts to host data stores, make sure you can connect to your cluster locally using kubectl. If you have your own data stores already setup, you just need to execute step 3 from below list.

1. Set up the data store corresponding to the test you wish to run. You can find Kubernetes scripts for all currently supported data stores in [.test-infra/kubernetes](https://github.com/apache/beam/tree/master/.test-infra/kubernetes).
1. In some cases, there is a setup script (*.sh). In other cases, you can just run ``kubectl create -f [scriptname]`` to create the data store.
1. Convention dictates there will be:
1. A core yml script for the data store itself, plus a `NodePort` service. The `NodePort` service opens a port to the data store for anyone who connects to the Kubernetes cluster's machines.
1. A separate script, called for-local-dev, which sets up a LoadBalancer service.
1. A yml script for the data store itself, plus a `NodePort` service. The `NodePort` service opens a port to the data store for anyone who connects to the Kubernetes cluster's machines from within same subnetwork. Such scripts are typically useful when running the scripts on Minikube Kubernetes Engine.
1. A separate script, with LoadBalancer service. Such service will expose an _external ip_ for the datastore. Such scripts are needed when external access is required (eg. on Jenkins).
1. Examples:
1. For JDBC, you can set up Postgres: `kubectl create -f .test-infra/kubernetes/postgres/postgres.yml`
1. For Elasticsearch, you can run the setup script: `bash .test-infra/kubernetes/elasticsearch/setup.sh`
1. Determine the IP address of the service:
1. NodePort service: `kubectl get pods -l 'component=elasticsearch' -o jsonpath={.items[0].status.podIP}`
1. LoadBalancer service:` kubectl get svc elasticsearch-external -o jsonpath='{.status.loadBalancer.ingress[0].ip}'`
1. Run the test using the instructions in the class (e.g. see the instructions in JdbcIOIT.java)
1. Run the test using `integrationTest` gradle task and the instructions in the test class (e.g. see the instructions in JdbcIOIT.java).
1. Tell Kubernetes to delete the resources specified in the Kubernetes scripts:
1. JDBC: `kubectl delete -f .test-infra/kubernetes/postgres/postgres.yml`
1. Elasticsearch: `bash .test-infra/kubernetes/elasticsearch/teardown.sh`

##### integrationTest Task {#integration-test-task}

Since `performanceTest` task involved running PerfkitBenchmarker, we can't use it to run the tests manually. For such purposes a more "low-level" task called `integrationTest` was introduced.


Example usage on Cloud Dataflow runner:

```
./gradlew integrationTest -p sdks/java/io/hadoop-input-format -DintegrationTestPipelineOptions='["--project=GOOGLE_CLOUD_PROJECT", "--tempRoot=GOOGLE_STORAGE_BUCKET", "--numberOfRecords=1000", "--postgresPort=5432", "--postgresServerName=SERVER_NAME", "--postgresUsername=postgres", "--postgresPassword=PASSWORD", "--postgresDatabaseName=postgres", "--postgresSsl=false", "--runner=TestDataflowRunner"]' -DintegrationTestRunner=dataflow --tests=org.apache.beam.sdk.io.hadoop.inputformat.HadoopInputFormatIOIT
```

Example usage on HDFS filesystem and Direct runner:

NOTE: Below setup will only work when /etc/hosts file contains entries with hadoop namenode and hadoop datanodes external IPs. Please see explanation in: [Small Cluster config file](https://github.com/apache/beam/blob/master/.test-infra/kubernetes/hadoop/SmallITCluster/pkb-config.yml) and [Large Cluster config file](https://github.com/apache/beam/blob/master/.test-infra/kubernetes/hadoop/LargeITCluster/pkb-config.yml).

```
export HADOOP_USER_NAME=root
./gradlew integrationTest -p sdks/java/io/file-based-io-tests -DintegrationTestPipelineOptions='["--numberOfRecords=1000", "--filenamePrefix=hdfs://HDFS_NAMENODE:9000/XMLIOIT", "--hdfsConfiguration=[{\"fs.defaultFS\":\"hdfs://HDFS_NAMENODE:9000\",\"dfs.replication\":1,\"dfs.client.use.datanode.hostname\":\"true\" }]" ]' -DintegrationTestRunner=direct -Dfilesystem=hdfs --tests org.apache.beam.sdk.io.xml.XmlIOIT
```

Parameter descriptions:


<table class="table">
<thead>
<tr>
<td>
<strong>Option</strong>
</td>
<td>
<strong>Function</strong>
</td>
</tr>
</thead>
<tbody>
<tr>
<td>-p sdks/java/io/file-based-io-tests/
</td>
<td>Specifies the project submodule of the I/O to test.
</td>
</tr>
<tr>
<td>-DintegrationTestPipelineOptions
</td>
<td>Passes pipeline options directly to the test being run.
</td>
</tr>
<tr>
<td>-DintegrationTestRunner
</td>
<td>Runner to be used for running the test. Currently possible options are: direct, dataflow.
</td>
</tr>
<tr>
<td>-Dfilesystem
</td>
<td>(optional, where applicable) Filesystem to be used to run the test. Currently possible options are: gcs, hdfs, s3. If not provided, local filesystem will be used.
</td>
</tr>
<tr>
<td>--tests
</td>
<td>Specifies the test to be run (fully qualified reference to class/test method).
</td>
</tr>
</tbody>
</table>

#### Running Integration Tests on Pull Requests {#running-on-pull-requests}

Thanks to [ghprb](https://github.com/janinko/ghprb) plugin it is possible to run Jenkins jobs when specific phrase is typed in a Github Pull Request's comment. Integration tests that have Jenkins job defined can be triggered this way. You can run integration tests using these phrases:

<table class="table">
<thead>
<tr>
<td>
<strong>Test</strong>
</td>
<td>
<strong>Phrase</strong>
</td>
</tr>
</thead>
<tbody>
<tr>
<td>JdbcIOIT
</td>
<td>Run Java JdbcIO Performance Test
</td>
</tr>
<tr>
<td>MongoDBIOIT
</td>
<td>Run Java MongoDBIO Performance Test
</td>
</tr>
<tr>
<td>HadoopInputFormatIOIT
</td>
<td>Run Java HadoopInputFormatIO Performance Test
</td>
</tr>
<tr>
<td>TextIO - local filesystem
</td>
<td>Run Java TextIO Performance Test
</td>
</tr>
<tr>
<td>TextIO - HDFS
</td>
<td>Run Java TextIO Performance Test HDFS
</td>
</tr>
<tr>
<td>Compressed TextIO - local filesystem
</td>
<td>Run Java CompressedTextIO Performance Test
</td>
</tr>
<tr>
<td>Compressed TextIO - HDFS
</td>
<td>Run Java CompressedTextIO Performance Test HDFS
</td>
</tr>
<tr>
<td>AvroIO - local filesystem
</td>
<td>Run Java AvroIO Performance Test
</td>
</tr>
<tr>
<td>AvroIO - HDFS
</td>
<td>Run Java AvroIO Performance Test HDFS
</td>
</tr>
<tr>
<td>TFRecordIO - local filesystem
</td>
<td>Run Java TFRecordIO Performance Test
</td>
</tr>
<tr>
<td>ParquetIO - local filesystem
</td>
<td>Run Java ParquetIO Performance Test
</td>
</tr>
<tr>
<td>XmlIO - local filesystem
</td>
<td>Run Java XmlIO Performance Test
</td>
</tr>
<tr>
<td>XmlIO - HDFS
</td>
<td>Run Java XmlIO Performance Test on HDFS
</td>
</tr>
</tbody>
</table>

Every job definition can be found in [.test-infra/jenkins](https://github.com/apache/beam/tree/master/.test-infra/jenkins).
If you modified/added new Jenkins job definitions in your Pull Request, run the seed job before running the integration test (comment: "Run seed job").

### Performance testing dashboard {#performance-testing-dashboard}

We measure the performance of IOITs by gathering test execution times from Jenkins jobs that run periodically. The consequent results are stored in a database (BigQuery), therefore we can display them in a form of plots.

The dashboard gathering all the results is available here: [Performance Testing Dashboard](https://apache-beam-testing.appspot.com/explore?dashboard=5755685136498688)

### Implementing Integration Tests {#implementing-integration-tests}

Expand All @@ -268,7 +453,6 @@ There are three components necessary to implement an integration test:

These three pieces are discussed in detail below.


#### Test Code {#test-code}

These are the conventions used by integration testing code:
Expand All @@ -295,12 +479,11 @@ As discussed in [Integration tests, data stores, and Kubernetes](#integration-te
If you would like help with this or have other questions, contact the Beam dev@ mailing list and the community may be able to assist you.

Guidelines for creating a Beam data store Kubernetes script:
1. **You must only provide access to the data store instance via a `NodePort` service.**
* This is a requirement for security, since it means that only the local network has access to the data store. This is particularly important since many data stores don't have security on by default, and even if they do, their passwords will be checked in to our public Github repo.

1. **You should define two Kubernetes scripts.**
* This is the best known way to implement item #1.
* The first script will contain the main datastore instance script (`StatefulSet`) plus a `NodePort` service exposing the data store. This will be the script run by the Beam Jenkins continuous integration server.
* The second script will define a `LoadBalancer` service, used for local development if the Kubernetes cluster is on another network. This file's name is usually suffixed with '-for-local-dev'.
* The second script will define an additional `LoadBalancer` service, used to expose an external IP address to the data store if the Kubernetes cluster is on another network. This file's name is usually suffixed with '-for-local-dev'.
1. **You must ensure that pods are recreated after crashes.**
* If you use a `pod` directly, it will not be recreated if the pod crashes or something causes the cluster to move the container for your pod.
* In most cases, you'll want to use `StatefulSet` as it supports persistent disks that last between restarts, and having a stable network identifier associated with the pod using a particular persistent disk. `Deployment` and `ReplicaSet` are also possibly useful, but likely in fewer scenarios since they do not have those features.
Expand Down

0 comments on commit a9b0c6b

Please sign in to comment.