Skip to content

Commit

Permalink
[BEAM-3217] Post code review changes #1
Browse files Browse the repository at this point in the history
  • Loading branch information
lgajowy committed Jan 9, 2018
1 parent 138b0e0 commit 421aabd
Show file tree
Hide file tree
Showing 7 changed files with 198 additions and 37 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,74 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

import common_job_properties

// This job runs the file-based IOs performance tests on PerfKit Benchmarker.
job('beam_PerformanceTests_HadoopInputFormatIO_IT') {
description('Runs PerfKit tests for HadoopInputFormatIO.')

// Set default Beam job properties.
common_job_properties.setTopLevelMainJobProperties(delegate)

// Allows triggering this build against pull requests.
common_job_properties.enablePhraseTriggeringFromPullRequest(
delegate,
'Java HadoopInputFormatIO Performance Test',
'Run Java HadoopInputFormatIO Performance Test')

// Run job in postcommit every 6 hours, don't trigger every push, and
// don't email individual committers.
common_job_properties.setPostCommit(
delegate,
'0 */6 * * *',
false,
'commits@beam.apache.org',
false)

def pipelineArgs = [
project: 'apache-beam-testing',
tempRoot: 'gs://temp-storage-for-perf-tests',
postgresPort: '5432',
numberOfRecords: '600000'
]
def pipelineArgList = []
pipelineArgs.each({
key, value -> pipelineArgList.add("\"--$key=$value\"")
})
def pipelineArgsJoined = "[" + pipelineArgList.join(',') + "]"

def argMap = [
beam_it_timeout: '1200',
benchmarks: 'beam_integration_benchmark',
beam_it_profile: 'io-it',
beam_prebuilt: 'true',
beam_sdk: 'java',
beam_it_module: 'sdks/java/io/hadoop/input-format',
beam_it_class: "org.apache.beam.sdk.io.hadoop.inputformat.HadoopInputFormatIOIT",
beam_it_options: pipelineArgsJoined,
beam_kubernetes_scripts: makePathAbsolute('.test-infra/kubernetes/postgres/postgres.yml'),
beam_options_config_file: makePathAbsolute('.test-infra/kubernetes/postgres/pkb-config.yml'),
bigquery_table: 'beam_performance.HadoopInputFormatIOIT_pkb_results'
]

common_job_properties.buildPerformanceTest(delegate, argMap)
}

static def makePathAbsolute(String path) {
return '"$WORKSPACE/src/' + path + '"'
}
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,7 @@ public static PGSimpleDataSource getPostgresDataSource(IOTestPipelineOptions opt
return dataSource;
}

public static void createDataTable(DataSource dataSource, String tableName)
public static void createTable(DataSource dataSource, String tableName)
throws SQLException {
try (Connection connection = dataSource.getConnection()) {
try (Statement statement = connection.createStatement()) {
Expand All @@ -51,7 +51,7 @@ public static void createDataTable(DataSource dataSource, String tableName)
}
}

public static void cleanUpDataTable(DataSource dataSource, String tableName)
public static void deleteTable(DataSource dataSource, String tableName)
throws SQLException {
if (tableName != null) {
try (Connection connection = dataSource.getConnection();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -97,7 +97,7 @@ public void processElement(ProcessContext c) {
private static final Map<Integer, String> EXPECTED_HASHES = ImmutableMap.of(
1000, "7d94d63a41164be058a9680002914358",
100_000, "c7cbddb319209e200f1c5eebef8fe960",
1_000_000, "205893d26b6b9753e3df62d52ad419f6"
600_000, "e2add2f680de9024e9bc46cd3912545e"
);

/**
Expand Down
109 changes: 91 additions & 18 deletions sdks/java/io/hadoop/input-format/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -47,20 +47,16 @@
</profile>

<!--
This profile invokes PerfKitBenchmarker, which does benchmarking of
the IO ITs. The arguments passed to it allow it to invoke mvn again
with the desired benchmark.
This profile invokes PerfKitBenchmarker, which does benchmarking of
the IO ITs. The arguments passed to it allow it to invoke mvn again
with the desired benchmark.

To invoke this, run:
To invoke this, run:

mvn verify -Dio-it-suite -pl sdks/java/io/file-based-io-tests
-DpkbLocation="path-to-pkb.py" \
-DintegrationTestPipelineOptions='["&ndash;&ndash;numberOfRecords=100000"]' \
-DioItClass=file-based IO IT class, eg. org.apache.beam.sdk.io.hadoop.input-format.HadoopInputFormatIOIT

For DirectRunner, please use -DforceDirectRunner=true argument
For other runners please check doc in BEAM-3060 and https://beam.apache.org/documentation/io/testing/
-->
mvn verify -Dio-it-suite -pl sdks/java/io/hadoop/input-format
-DpkbLocation="path-to-pkb.py" \
-DintegrationTestPipelineOptions='["-tempRoot=gs://bucket/staging"]'
-->
<profile>
<id>io-it-suite</id>
<activation>
Expand Down Expand Up @@ -119,19 +115,96 @@
<argument>${pkbBeamRunnerOption}</argument>
<!-- specific to this IO -->
<argument>-beam_it_module=sdks/java/io/hadoop/input-format</argument>
<argument>-beam_it_class=${ioItClass}</argument>
<argument>-beam_it_class=org.apache.beam.sdk.io.hadoop.inputformat.HadoopInputFormatIOIT</argument>
<argument>-beam_options_config_file=${beamRootProjectDir}/.test-infra/kubernetes/postgres/pkb-config.yml</argument>
<argument>-beam_kubernetes_scripts=${beamRootProjectDir}/.test-infra/kubernetes/postgres/postgres.yml</argument>
<!-- arguments typically defined by user -->
<argument>-beam_it_options=${integrationTestPipelineOptions}</argument>
</arguments>
</configuration>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-surefire-plugin</artifactId>
<version>${surefire-plugin.version}</version>
<configuration>
<skipTests>true</skipTests>
</configuration>
</plugin>
</plugins>
</build>
</profile>
<!--
io-it-suite-local overrides part of io-it-suite, allowing users to run tests
when they are on a separate network from the kubernetes cluster by
creating a LoadBalancer service.
-->
<profile>
<id>io-it-suite-local</id>
<activation><property><name>io-it-suite-local</name></property></activation>
<properties>
<!-- This is based on the location of the current pom relative to the root
See discussion in BEAM-2460 -->
<beamRootProjectDir>${project.parent.parent.parent.parent.parent.basedir}</beamRootProjectDir>
</properties>
<build>
<plugins>
<plugin>
<groupId>org.codehaus.gmaven</groupId>
<artifactId>groovy-maven-plugin</artifactId>
<version>${groovy-maven-plugin.version}</version>
<executions>
<execution>
<id>find-supported-python-for-compile</id>
<phase>initialize</phase>
<goals>
<goal>execute</goal>
</goals>
<configuration>
<source>${beamRootProjectDir}/sdks/python/findSupportedPython.groovy</source>
</configuration>
</execution>
</executions>
</plugin>
<plugin>
<groupId>org.codehaus.mojo</groupId>
<artifactId>exec-maven-plugin</artifactId>
<version>${maven-exec-plugin.version}</version>
<executions>
<execution>
<phase>verify</phase>
<goals>
<goal>exec</goal>
</goals>
</execution>
</executions>
<configuration>
<executable>${python.interpreter.bin}</executable>
<arguments>
<argument>${pkbLocation}</argument>
<argument>-benchmarks=beam_integration_benchmark</argument>
<argument>-beam_it_profile=io-it</argument>
<argument>-beam_location=${beamRootProjectDir}</argument>
<argument>-beam_prebuilt=true</argument>
<argument>-beam_sdk=java</argument>
<argument>-kubeconfig=${kubeconfig}</argument>
<argument>-kubectl=${kubectl}</argument>
<!-- runner overrides, controlled via forceDirectRunner -->
<argument>${pkbBeamRunnerProfile}</argument>
<argument>${pkbBeamRunnerOption}</argument>
<!-- specific to this IO -->
<argument>-beam_it_module=sdks/java/io/hadoop/input-format</argument>
<argument>-beam_it_class=org.apache.beam.sdk.io.hadoop.inputformat.HadoopInputFormatIOIT</argument>
<argument>-beam_options_config_file=${beamRootProjectDir}/.test-infra/kubernetes/postgres/pkb-config-local.yml</argument>
<argument>-beam_kubernetes_scripts=${beamRootProjectDir}/.test-infra/kubernetes/postgres/postgres.yml,${beamRootProjectDir}/.test-infra/kubernetes/postgres/postgres-service-for-local-dev.yml</argument>
<!-- arguments typically defined by user -->
<argument>-beam_it_options=${integrationTestPipelineOptions}</argument>
<!--
optional array of key=value items. It will be passed to
target mvn command by pkb. eg. -DpkbExtraProperties='["filesystem=local"]'
-->
<argument>-beam_extra_mvn_properties=${pkbExtraProperties}</argument>
</arguments>
</configuration>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-surefire-plugin</artifactId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@

import static org.apache.beam.sdk.io.common.TestRow.DeterministicallyConstructTestRowFn;
import static org.apache.beam.sdk.io.common.TestRow.SelectNameFn;
import static org.apache.beam.sdk.io.common.TestRow.getExpectedHashForRowCount;
import static org.apache.beam.sdk.io.hadoop.inputformat.TestRowDBWritable.PrepareStatementFromTestRow;

import java.sql.SQLException;
Expand Down Expand Up @@ -49,8 +50,22 @@
import org.junit.Test;
import org.postgresql.ds.PGSimpleDataSource;


/**
* IOIT for {@link org.apache.beam.sdk.io.hadoop.inputformat.HadoopInputFormatIO}.
* A test of {@link org.apache.beam.sdk.io.hadoop.inputformat.HadoopInputFormatIO}
* on an independent postgres instance.
*
* <p>This test requires a running instance of Postgres. Pass in connection information using
* PipelineOptions:
* <pre>
* mvn -e -Pio-it verify -pl sdks/java/io/hadoop/input-format/ -DintegrationTestPipelineOptions='[
* "--postgresServerName=1.2.3.4",
* "--postgresUsername=postgres",
* "--postgresDatabaseName=myfancydb",
* "--postgresPassword=mypass",
* "--postgresSsl=false",
* "--numberOfRecords=1000" ]'
* </pre>
*/
public class HadoopInputFormatIOIT {

Expand All @@ -75,7 +90,7 @@ public static void setUp() throws SQLException {
numberOfRows = options.getNumberOfRecords();
tableName = DatabaseTestHelper.getTestTableName("HadoopInputFormatIOIT");

DatabaseTestHelper.createDataTable(dataSource, tableName);
DatabaseTestHelper.createTable(dataSource, tableName);
setupHadoopConfiguration(options);
}

Expand All @@ -101,32 +116,31 @@ private static void setupHadoopConfiguration(IOTestPipelineOptions options) {

@AfterClass
public static void tearDown() throws SQLException {
DatabaseTestHelper.cleanUpDataTable(dataSource, tableName);
DatabaseTestHelper.deleteTable(dataSource, tableName);
}

@Test
public void writeThenReadUsingDBInputFormat() {
public void readUsingHadoopInputFormat() {
writePipeline.apply("Generate sequence", GenerateSequence.from(0).to(numberOfRows))
.apply("Produce db rows", ParDo.of(new DeterministicallyConstructTestRowFn()))
.apply(JdbcIO.<TestRow>write()
.apply("Prevent fusion before writing", Reshuffle.<TestRow>viaRandomKey())
.apply("Write using JDBCIO", JdbcIO.<TestRow>write()
.withDataSourceConfiguration(JdbcIO.DataSourceConfiguration.create(dataSource))
.withStatement(String.format("insert into %s values(?, ?)", tableName))
.withPreparedStatementSetter(new PrepareStatementFromTestRow()));


writePipeline.run().waitUntilFinish();

PCollection<String> consolidatedHashcode = readPipeline
.apply("Read using DBInputFormat", HadoopInputFormatIO
.apply("Read using HadoopInputFormat", HadoopInputFormatIO
.<LongWritable, TestRowDBWritable>read()
.withConfiguration(hadoopConfiguration.get()))
.apply("Get values only", Values.<TestRowDBWritable>create())
.apply("Values as string", ParDo.of(new SelectNameFn()))
.apply("Calculate hashcode", Combine.globally(new HashingFn()))
.apply(Reshuffle.<String>viaRandomKey());
.apply("Calculate hashcode", Combine.globally(new HashingFn()));

PAssert.thatSingleton(consolidatedHashcode)
.isEqualTo(TestRow.getExpectedHashForRowCount(numberOfRows.intValue()));
.isEqualTo(getExpectedHashForRowCount(numberOfRows.intValue()));

readPipeline.run().waitUntilFinish();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -87,12 +87,12 @@ public static void setup() throws SQLException, ParseException {
dataSource = getPostgresDataSource(options);

tableName = DatabaseTestHelper.getTestTableName("IT");
DatabaseTestHelper.createDataTable(dataSource, tableName);
DatabaseTestHelper.createTable(dataSource, tableName);
}

@AfterClass
public static void tearDown() throws SQLException {
DatabaseTestHelper.cleanUpDataTable(dataSource, tableName);
DatabaseTestHelper.deleteTable(dataSource, tableName);
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -114,14 +114,14 @@ public static void startDatabase() throws Exception {

readTableName = DatabaseTestHelper.getTestTableName("UT_READ");

DatabaseTestHelper.createDataTable(dataSource, readTableName);
DatabaseTestHelper.createTable(dataSource, readTableName);
addInitialData(dataSource, readTableName);
}

@AfterClass
public static void shutDownDatabase() throws Exception {
try {
DatabaseTestHelper.cleanUpDataTable(dataSource, readTableName);
DatabaseTestHelper.deleteTable(dataSource, readTableName);
} finally {
if (derbyServer != null) {
derbyServer.shutdown();
Expand Down Expand Up @@ -255,7 +255,7 @@ public void testWrite() throws Exception {
final long rowsToAdd = 1000L;

String tableName = DatabaseTestHelper.getTestTableName("UT_WRITE");
DatabaseTestHelper.createDataTable(dataSource, tableName);
DatabaseTestHelper.createTable(dataSource, tableName);
try {
ArrayList<KV<Integer, String>> data = new ArrayList<>();
for (int i = 0; i < rowsToAdd; i++) {
Expand Down Expand Up @@ -291,7 +291,7 @@ public void setParameters(
}
}
} finally {
DatabaseTestHelper.cleanUpDataTable(dataSource, tableName);
DatabaseTestHelper.deleteTable(dataSource, tableName);
}
}

Expand Down

0 comments on commit 421aabd

Please sign in to comment.