Skip to content

Commit

Permalink
[BEAM-4496] Fix apache#2 branch fetch failure on job_PostCommit_Websi…
Browse files Browse the repository at this point in the history
…te_Publish
  • Loading branch information
Alan Myrvold committed Oct 1, 2018
1 parent fc38698 commit d9ca6e3
Show file tree
Hide file tree
Showing 28 changed files with 932 additions and 563 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -16,13 +16,13 @@
# limitations under the License.
#

import dependency_check.version_comparer as version_comparer
import logging
import os.path
import re
import requests
import sys
import traceback
import version_comparer

from datetime import datetime
from dependency_check.bigquery_client_utils import BigQueryClientUtils
Expand Down
2 changes: 1 addition & 1 deletion .test-infra/jenkins/job_PostCommit_Website_Publish.groovy
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ import PostcommitJobBuilder


// This job builds and publishes the website into the asf-site branch of the beam repo.
PostcommitJobBuilder.postCommitJob('beam_PostCommit_Website_Publish', '',
PostcommitJobBuilder.postCommitJob('beam_PostCommit_Website_Publish', 'Run Website Publish',
'Website Publish', this) {

description('Publish generated website content into asf-site branch for hosting.')
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,7 @@
import org.apache.beam.sdk.options.PipelineOptionsFactory;
import org.apache.beam.sdk.options.StreamingOptions;
import org.apache.beam.sdk.options.Validation;
import org.apache.beam.sdk.testing.PAssert;
import org.apache.beam.sdk.transforms.Count;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.Filter;
Expand All @@ -60,6 +61,7 @@
import org.apache.beam.sdk.transforms.Partition;
import org.apache.beam.sdk.transforms.Partition.PartitionFn;
import org.apache.beam.sdk.transforms.SerializableFunction;
import org.apache.beam.sdk.transforms.Sum;
import org.apache.beam.sdk.transforms.Top;
import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
Expand Down Expand Up @@ -431,6 +433,17 @@ public interface Options extends ExampleOptions, ExampleBigQueryTableOptions, St

void setOutputToBigQuery(Boolean value);

@Description("Whether to send output to checksum Transform.")
@Default.Boolean(true)
Boolean getOutputToChecksum();

void setOutputToChecksum(Boolean value);

@Description("Expected result of the checksum transform.")
Long getExpectedChecksum();

void setExpectedChecksum(Long value);

@Description("Whether output to Cloud Datastore")
@Default.Boolean(false)
Boolean getOutputToDatastore();
Expand All @@ -449,8 +462,7 @@ public interface Options extends ExampleOptions, ExampleBigQueryTableOptions, St
void setOutputProject(String value);
}

public static void main(String[] args) throws IOException {
Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
public static void runAutocompletePipeline(Options options) throws IOException {

options.setBigQuerySchema(FormatForBigquery.getSchema());
ExampleUtils exampleUtils = new ExampleUtils(options);
Expand Down Expand Up @@ -506,10 +518,35 @@ public static void main(String[] args) throws IOException {
: BigQueryIO.Write.WriteDisposition.WRITE_TRUNCATE));
}

if (options.getOutputToChecksum()) {
PCollection<Long> checksum =
toWrite
.apply(
ParDo.of(
new DoFn<KV<String, List<CompletionCandidate>>, Long>() {
@ProcessElement
public void process(ProcessContext c) {
KV<String, List<CompletionCandidate>> elm = c.element();
Long listHash =
c.element().getValue().stream().mapToLong(cc -> cc.hashCode()).sum();
c.output(Long.valueOf(elm.getKey().hashCode()) + listHash);
}
}))
.apply(Sum.longsGlobally());

PAssert.that(checksum).containsInAnyOrder(options.getExpectedChecksum());
}

// Run the pipeline.
PipelineResult result = p.run();

// ExampleUtils will try to cancel the pipeline and the injector before the program exists.
exampleUtils.waitToFinish(result);
}

public static void main(String[] args) throws IOException {
Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);

runAutocompletePipeline(options);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,59 @@
/*
* 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.
*/
package org.apache.beam.examples.complete;

import org.apache.beam.examples.complete.AutoComplete.Options;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
import org.apache.beam.sdk.testing.TestPipeline;
import org.apache.beam.sdk.testing.TestPipelineOptions;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.JUnit4;

/** Run integration tests for a basic AutoComplete pipeline. */
@RunWith(JUnit4.class)
public class AutoCompleteIT {

public static final String DEFAULT_INPUT =
"gs://apache-beam-samples/shakespeare/kinglear-hashtag.txt";

public static final Long DEFAULT_INPUT_CHECKSUM = -25447108232L;

/** Options for the Autocomplete Integration test. */
public interface AutoCompleteITOptions extends TestPipelineOptions, Options {}

@BeforeClass
public static void setUp() {
PipelineOptionsFactory.register(TestPipelineOptions.class);
}

@Test
public void testE2EAutoComplete() throws Exception {
AutoCompleteITOptions options =
TestPipeline.testingPipelineOptions().as(AutoCompleteITOptions.class);

options.setInputFile(DEFAULT_INPUT);
options.setOutputToBigQuery(false);
options.setOutputToDatastore(false);
options.setOutputToChecksum(true);
options.setExpectedChecksum(DEFAULT_INPUT_CHECKSUM);

AutoComplete.runAutocompletePipeline(options);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,50 +18,69 @@
package org.apache.beam.examples.cookbook;

import com.google.api.services.bigquery.model.TableRow;
import java.util.List;
import com.google.common.collect.ImmutableList;
import org.apache.beam.examples.cookbook.BigQueryTornadoes.ExtractTornadoesFn;
import org.apache.beam.examples.cookbook.BigQueryTornadoes.FormatCountsFn;
import org.apache.beam.sdk.transforms.DoFnTester;
import org.apache.beam.sdk.testing.PAssert;
import org.apache.beam.sdk.testing.TestPipeline;
import org.apache.beam.sdk.testing.ValidatesRunner;
import org.apache.beam.sdk.transforms.Create;
import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.values.KV;
import org.hamcrest.CoreMatchers;
import org.junit.Assert;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.TypeDescriptor;
import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.runner.RunWith;
import org.junit.runners.JUnit4;

/** Test case for {@link BigQueryTornadoes}. */
@RunWith(JUnit4.class)
public class BigQueryTornadoesTest {
@Rule public TestPipeline p = TestPipeline.create();

@Test
public void testExtractTornadoes() throws Exception {
@Category(ValidatesRunner.class)
public void testExtractTornadoes() {
TableRow row = new TableRow().set("month", "6").set("tornado", true);
DoFnTester<TableRow, Integer> extractWordsFn = DoFnTester.of(new ExtractTornadoesFn());
Assert.assertThat(extractWordsFn.processBundle(row), CoreMatchers.hasItems(6));
PCollection<TableRow> input = p.apply(Create.of(ImmutableList.of(row)));
PCollection<Integer> result = input.apply(ParDo.of(new ExtractTornadoesFn()));
PAssert.that(result).containsInAnyOrder(6);
p.run().waitUntilFinish();
}

@Test
public void testNoTornadoes() throws Exception {
@Category(ValidatesRunner.class)
public void testNoTornadoes() {
TableRow row = new TableRow().set("month", 6).set("tornado", false);
DoFnTester<TableRow, Integer> extractWordsFn = DoFnTester.of(new ExtractTornadoesFn());
Assert.assertTrue(extractWordsFn.processBundle(row).isEmpty());
PCollection<TableRow> inputs = p.apply(Create.of(ImmutableList.of(row)));
PCollection<Integer> result = inputs.apply(ParDo.of(new ExtractTornadoesFn()));
PAssert.that(result).empty();
p.run().waitUntilFinish();
}

@Test
@SuppressWarnings({"rawtypes", "unchecked"})
public void testFormatCounts() throws Exception {
DoFnTester<KV<Integer, Long>, TableRow> formatCountsFn = DoFnTester.of(new FormatCountsFn());
KV empty[] = {};
List<TableRow> results = formatCountsFn.processBundle(empty);
Assert.assertTrue(results.isEmpty());
KV input[] = {KV.of(3, 0L), KV.of(4, Long.MAX_VALUE), KV.of(5, Long.MIN_VALUE)};
results = formatCountsFn.processBundle(input);
Assert.assertEquals(3, results.size());
Assert.assertEquals(3, results.get(0).get("month"));
Assert.assertEquals(0L, results.get(0).get("tornado_count"));
Assert.assertEquals(4, results.get(1).get("month"));
Assert.assertEquals(Long.MAX_VALUE, results.get(1).get("tornado_count"));
Assert.assertEquals(5, results.get(2).get("month"));
Assert.assertEquals(Long.MIN_VALUE, results.get(2).get("tornado_count"));
@Category(ValidatesRunner.class)
public void testEmpty() {
PCollection<KV<Integer, Long>> inputs =
p.apply(Create.empty(new TypeDescriptor<KV<Integer, Long>>() {}));
PCollection<TableRow> result = inputs.apply(ParDo.of(new FormatCountsFn()));
PAssert.that(result).empty();
p.run().waitUntilFinish();
}

@Test
@Category(ValidatesRunner.class)
public void testFormatCounts() {
PCollection<KV<Integer, Long>> inputs =
p.apply(Create.of(KV.of(3, 0L), KV.of(4, Long.MAX_VALUE), KV.of(5, Long.MIN_VALUE)));
PCollection<TableRow> result = inputs.apply(ParDo.of(new FormatCountsFn()));
PAssert.that(result)
.containsInAnyOrder(
new TableRow().set("month", 3).set("tornado_count", 0),
new TableRow().set("month", 4).set("tornado_count", Long.MAX_VALUE),
new TableRow().set("month", 5).set("tornado_count", Long.MIN_VALUE));
p.run().waitUntilFinish();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -295,3 +295,4 @@ PUBSUB_SUBSCRIPTION='wordstream-python-sub2'
# Mobile Gaming Configurations
DATASET='beam_postrelease_mobile_gaming'
USERSCORE_OUTPUT_PREFIX='python-userscore_result'
GAME_INPUT_DATA='gs://dataflow-samples/game/5000_gaming_data.csv'
Original file line number Diff line number Diff line change
Expand Up @@ -81,7 +81,7 @@ function verify_userscore_direct() {
--output=$output_file_name \
--project=$PROJECT_ID \
--dataset=$DATASET \
--input=gs://$BUCKET_NAME/5000_gaming_data.csv
--input=$GAME_INPUT_DATA

verify_user_score "direct"
}
Expand All @@ -103,7 +103,7 @@ function verify_userscore_dataflow() {
--runner=DataflowRunner \
--temp_location=gs://$BUCKET_NAME/temp/ \
--sdk_location=$BEAM_PYTHON_SDK \
--input=gs://$BUCKET_NAME/5000_gaming_data.csv \
--input=$GAME_INPUT_DATA \
--output=gs://$BUCKET_NAME/$output_file_name

verify_user_score "dataflow"
Expand All @@ -122,7 +122,7 @@ function verify_hourlyteamscore_direct() {
python -m apache_beam.examples.complete.game.hourly_team_score \
--project=$PROJECT_ID \
--dataset=$DATASET \
--input=gs://$BUCKET_NAME/5000_gaming_data.csv \
--input=$GAME_INPUT_DATA \
--table="hourly_team_score_python_direct"

verify_hourly_team_score "direct"
Expand All @@ -145,7 +145,7 @@ function verify_hourlyteamscore_dataflow() {
--runner=DataflowRunner \
--temp_location=gs://$BUCKET_NAME/temp/ \
--sdk_location $BEAM_PYTHON_SDK \
--input=gs://$BUCKET_NAME/5000_gaming_data.csv \
--input=$GAME_INPUT_DATA \
--table="hourly_team_score_python_dataflow"

verify_hourly_team_score "dataflow"
Expand Down
1 change: 1 addition & 0 deletions runners/reference/java/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ configurations {
}

dependencies {
compile library.java.guava
compile library.java.hamcrest_library
shadow project(path: ":beam-model-pipeline", configuration: "shadow")
shadow project(path: ":beam-runners-core-construction-java", configuration: "shadow")
Expand Down
48 changes: 48 additions & 0 deletions runners/reference/job-server/build.gradle
Original file line number Diff line number Diff line change
@@ -0,0 +1,48 @@
/*
* 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 groovy.json.JsonOutput

apply plugin: org.apache.beam.gradle.BeamModulePlugin
applyJavaNature(
validateShadowJar: false,
shadowClosure: {
}
)

description = "Apache Beam :: Runners :: Reference :: Job Server"

apply plugin: "application"

mainClassName = "org.apache.beam.runners.direct.portable.job.ReferenceRunnerJobServer"

dependencies {
compile project(path: ":beam-runners-direct-java", configuration: "shadow")
compile project(path: ":beam-runners-java-fn-execution", configuration: "shadow")
compile library.java.slf4j_simple
}

run {
args = []
if (project.hasProperty('port'))
args += ["--port=${project.property('port')}"]

// Enable remote debugging.
jvmArgs = ["-Xdebug", "-Xrunjdwp:transport=dt_socket,server=y,suspend=n,address=5005"]
if (project.hasProperty("logLevel"))
jvmArgs += ["-Dorg.slf4j.simpleLogger.defaultLogLevel=${project.property('logLevel')}"]
}
2 changes: 1 addition & 1 deletion sdks/go/pkg/beam/options/jobopts/options.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,7 @@ var (
EnvironmentType = flag.String("environment_type", "DOCKER",
"Environment Type. Possible options are DOCKER and PROCESS.")

// EnvironmentType is the environment type to run the user code.
// EnvironmentConfig is the environment configuration for running the user code.
EnvironmentConfig = flag.String("environment_config",
"",
"Set environment configuration for running the user code.\n"+
Expand Down
Loading

0 comments on commit d9ca6e3

Please sign in to comment.