[jira] [Commented] (BEAM-371) Backport HDFS IO enhancements from Scio
[ https://issues.apache.org/jira/browse/BEAM-371?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15359937#comment-15359937 ] ASF GitHub Bot commented on BEAM-371: - Github user asfgit closed the pull request at: https://github.com/apache/incubator-beam/pull/485 > Backport HDFS IO enhancements from Scio > --- > > Key: BEAM-371 > URL: https://issues.apache.org/jira/browse/BEAM-371 > Project: Beam > Issue Type: Improvement > Components: sdk-java-extensions >Affects Versions: 0.1.0-incubating >Reporter: Neville Li >Assignee: Neville Li >Priority: Minor > Fix For: 0.2.0-incubating > > > Right now there is a {{beam-sdks-java-io-hdfs}} module but only > {{HDFSFileSource}} is implemented and there's a known issue with reading Avro > files. > https://github.com/GoogleCloudPlatform/DataflowJavaSDK/issues/102 > We at Spotify have implemented HDFS sinks, specialized source/sink for Avro > and simple authentication and would like to port it back to Beam. > https://github.com/apache/incubator-beam/pull/485 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (BEAM-371) Backport HDFS IO enhancements from Scio
[ https://issues.apache.org/jira/browse/BEAM-371?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15359939#comment-15359939 ] Daniel Halperin commented on BEAM-371: -- Thanks for the contribution, Neville! > Backport HDFS IO enhancements from Scio > --- > > Key: BEAM-371 > URL: https://issues.apache.org/jira/browse/BEAM-371 > Project: Beam > Issue Type: Improvement > Components: sdk-java-extensions >Affects Versions: 0.1.0-incubating >Reporter: Neville Li >Assignee: Neville Li >Priority: Minor > Fix For: 0.2.0-incubating > > > Right now there is a {{beam-sdks-java-io-hdfs}} module but only > {{HDFSFileSource}} is implemented and there's a known issue with reading Avro > files. > https://github.com/GoogleCloudPlatform/DataflowJavaSDK/issues/102 > We at Spotify have implemented HDFS sinks, specialized source/sink for Avro > and simple authentication and would like to port it back to Beam. > https://github.com/apache/incubator-beam/pull/485 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Closed] (BEAM-371) Backport HDFS IO enhancements from Scio
[ https://issues.apache.org/jira/browse/BEAM-371?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Daniel Halperin closed BEAM-371. Resolution: Fixed Fix Version/s: 0.2.0-incubating > Backport HDFS IO enhancements from Scio > --- > > Key: BEAM-371 > URL: https://issues.apache.org/jira/browse/BEAM-371 > Project: Beam > Issue Type: Improvement > Components: sdk-java-extensions >Affects Versions: 0.1.0-incubating >Reporter: Neville Li >Assignee: Neville Li >Priority: Minor > Fix For: 0.2.0-incubating > > > Right now there is a {{beam-sdks-java-io-hdfs}} module but only > {{HDFSFileSource}} is implemented and there's a known issue with reading Avro > files. > https://github.com/GoogleCloudPlatform/DataflowJavaSDK/issues/102 > We at Spotify have implemented HDFS sinks, specialized source/sink for Avro > and simple authentication and would like to port it back to Beam. > https://github.com/apache/incubator-beam/pull/485 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[1/2] incubator-beam git commit: Several improvements to HDFS/Hadoop interoperability
Repository: incubator-beam Updated Branches: refs/heads/master 69b0a48e8 -> c834ecd3d Several improvements to HDFS/Hadoop interoperability * handle NullWritable in WritableCoder * update Function handling in HDFSFileSource#splitIntoBundles * add AvroHDFSFileSource * add HDFSFileSink * add SimpleAuth HDFS IO Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/ed7dbb07 Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/ed7dbb07 Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/ed7dbb07 Branch: refs/heads/master Commit: ed7dbb07f41b3969c28a6096a385005ee3a1ff7f Parents: 69b0a48 Author: Neville LiAuthored: Mon Jun 27 15:51:17 2016 -0400 Committer: Dan Halperin Committed: Fri Jul 1 19:54:06 2016 -0700 -- sdks/java/io/hdfs/pom.xml | 24 ++ .../beam/sdk/io/hdfs/AvroHDFSFileSource.java| 145 ++ .../beam/sdk/io/hdfs/AvroWrapperCoder.java | 116 .../apache/beam/sdk/io/hdfs/HDFSFileSink.java | 277 +++ .../apache/beam/sdk/io/hdfs/HDFSFileSource.java | 40 +-- .../apache/beam/sdk/io/hdfs/WritableCoder.java | 9 +- .../SimpleAuthAvroHDFSFileSource.java | 84 ++ .../hdfs/simpleauth/SimpleAuthHDFSFileSink.java | 132 + .../simpleauth/SimpleAuthHDFSFileSource.java| 122 .../beam/sdk/io/hdfs/AvroWrapperCoderTest.java | 52 .../beam/sdk/io/hdfs/WritableCoderTest.java | 9 + 11 files changed, 989 insertions(+), 21 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ed7dbb07/sdks/java/io/hdfs/pom.xml -- diff --git a/sdks/java/io/hdfs/pom.xml b/sdks/java/io/hdfs/pom.xml index 2e427b1..42175d5 100644 --- a/sdks/java/io/hdfs/pom.xml +++ b/sdks/java/io/hdfs/pom.xml @@ -82,6 +82,30 @@ + com.google.http-client + google-http-client + + + + org.apache.avro + avro + + + + org.apache.avro + avro-mapred + ${avro.version} + hadoop2 + + + + org.mortbay.jetty + servlet-api + + + + + org.apache.hadoop hadoop-client 2.7.0 http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ed7dbb07/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/AvroHDFSFileSource.java -- diff --git a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/AvroHDFSFileSource.java b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/AvroHDFSFileSource.java new file mode 100644 index 000..9dc926b --- /dev/null +++ b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/AvroHDFSFileSource.java @@ -0,0 +1,145 @@ +/* + * 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.sdk.io.hdfs; + +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.util.CoderUtils; +import org.apache.beam.sdk.values.KV; + +import com.google.common.base.Function; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; + +import org.apache.avro.Schema; +import org.apache.avro.mapred.AvroKey; +import org.apache.avro.mapreduce.AvroJob; +import org.apache.avro.mapreduce.AvroKeyInputFormat; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.mapreduce.InputSplit; +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; + +import java.io.IOException; +import java.util.List; +import javax.annotation.Nullable; + +/** + * A {@code BoundedSource} for reading Avro files resident in a Hadoop filesystem. + * + * @param The type of the Avro records to be read from the source. + */ +public class AvroHDFSFileSource
[jira] [Commented] (BEAM-357) Build doesn't pass on windows
[ https://issues.apache.org/jira/browse/BEAM-357?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15359897#comment-15359897 ] Davor Bonaci commented on BEAM-357: --- [~romain.manni-bucau] seems to be on this with pending PRs. So, I'm going to reassign back. > Build doesn't pass on windows > - > > Key: BEAM-357 > URL: https://issues.apache.org/jira/browse/BEAM-357 > Project: Beam > Issue Type: Bug > Components: runner-flink, runner-spark, sdk-java-extensions >Reporter: Romain Manni-Bucau >Assignee: Jean-Baptiste Onofré > Fix For: 0.2.0-incubating > > Attachments: bean-win.diff > > -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (BEAM-389) DelegateCoder needs equals and hashCode
[ https://issues.apache.org/jira/browse/BEAM-389?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Davor Bonaci updated BEAM-389: -- Component/s: sdk-java-core > DelegateCoder needs equals and hashCode > --- > > Key: BEAM-389 > URL: https://issues.apache.org/jira/browse/BEAM-389 > Project: Beam > Issue Type: Bug > Components: sdk-java-core >Reporter: Pei He >Assignee: Pei He >Priority: Minor > > Currently, DelegateCoder inherit equals() and hashCode() from StandardCoder. > And, it makes DelegateCoder.of(VarIntCoder) equal to > DelegateCoder.of(BigEndianIntegerCoder). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[GitHub] incubator-beam pull request #582: Replace PubsubIO and injector with TextIO ...
GitHub user peihe opened a pull request: https://github.com/apache/incubator-beam/pull/582 Replace PubsubIO and injector with TextIO in AutoComplete and TriggerExample You can merge this pull request into a Git repository by running: $ git pull https://github.com/peihe/incubator-beam streaming-examples-5 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/incubator-beam/pull/582.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #582 commit 0408790eccfe440458519e9018d93b038d4c66d7 Author: Pei HeDate: 2016-07-02T00:45:37Z Replace PubsubIO and injector with TextIO in AutoComplete and TriggerExample --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] incubator-beam pull request #562: Replace PubsubIO and injector with TextIO ...
Github user asfgit closed the pull request at: https://github.com/apache/incubator-beam/pull/562 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[2/2] incubator-beam git commit: Replace PubsubIO and injector with TextIO in beam-examples
Replace PubsubIO and injector with TextIO in beam-examples Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/40d8072c Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/40d8072c Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/40d8072c Branch: refs/heads/master Commit: 40d8072c2d03dd80a7e644cb1587c765b1477d5a Parents: 85c36b8 Author: Pei HeAuthored: Wed Jun 29 13:01:10 2016 -0700 Committer: Dan Halperin Committed: Fri Jul 1 17:21:37 2016 -0700 -- .../examples/complete/StreamingWordExtract.java | 35 ++-- 1 file changed, 10 insertions(+), 25 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/40d8072c/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java -- diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java b/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java index aca4222..4ea199c 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java @@ -19,11 +19,10 @@ package org.apache.beam.examples.complete; import org.apache.beam.examples.common.DataflowExampleUtils; import org.apache.beam.examples.common.ExampleBigQueryTableOptions; -import org.apache.beam.examples.common.ExamplePubsubTopicOptions; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.io.BigQueryIO; -import org.apache.beam.sdk.io.PubsubIO; +import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.PipelineOptionsFactory; @@ -40,22 +39,14 @@ import java.util.ArrayList; /** * A streaming Dataflow Example using BigQuery output. * - * This pipeline example reads lines of text from a PubSub topic, splits each line + * This pipeline example reads lines of the input text file, splits each line * into individual words, capitalizes those words, and writes the output to * a BigQuery table. * - * By default, the example will run a separate pipeline to inject the data from the default - * {@literal --inputFile} to the Pub/Sub {@literal --pubsubTopic}. It will make it available for - * the streaming pipeline to process. You may override the default {@literal --inputFile} with the - * file of your choosing. You may also set {@literal --inputFile} to an empty string, which will - * disable the automatic Pub/Sub injection, and allow you to use separate tool to control the input - * to this example. - * - * The example is configured to use the default Pub/Sub topic and the default BigQuery table - * from the example common package (there are no defaults for a general Dataflow pipeline). - * You can override them by using the {@literal --pubsubTopic}, {@literal --bigQueryDataset}, and - * {@literal --bigQueryTable} options. If the Pub/Sub topic or the BigQuery table do not exist, - * the example will try to create them. + * The example is configured to use the default BigQuery table from the example common package + * (there are no defaults for a general Dataflow pipeline). + * You can override them by using the {@literal --bigQueryDataset}, and {@literal --bigQueryTable} + * options. If the BigQuery table do not exist, the example will try to create them. * * The example will try to cancel the pipelines on the signal to terminate the process (CTRL-C) * and then exits. @@ -110,9 +101,8 @@ public class StreamingWordExtract { * * Inherits standard configuration options. */ - private interface StreamingWordExtractOptions - extends ExamplePubsubTopicOptions, ExampleBigQueryTableOptions { -@Description("Input file to inject to Pub/Sub topic") + private interface StreamingWordExtractOptions extends ExampleBigQueryTableOptions { +@Description("Path of the file to read from") @Default.String("gs://dataflow-samples/shakespeare/kinglear.txt") String getInputFile(); void setInputFile(String value); @@ -141,7 +131,7 @@ public class StreamingWordExtract { .append(options.getBigQueryTable()) .toString(); pipeline -.apply(PubsubIO.Read.topic(options.getPubsubTopic())) +.apply("ReadLines", TextIO.Read.from(options.getInputFile())) .apply(ParDo.of(new ExtractWords())) .apply(ParDo.of(new Uppercase())) .apply(ParDo.of(new StringToRowConverter())) @@ -150,12 +140,7 @@ public class StreamingWordExtract {
[1/2] incubator-beam git commit: Closes #562
Repository: incubator-beam Updated Branches: refs/heads/master 85c36b856 -> 69b0a48e8 Closes #562 Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/69b0a48e Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/69b0a48e Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/69b0a48e Branch: refs/heads/master Commit: 69b0a48e88abf7b8d4f747cc8f03e04472e3d796 Parents: 85c36b8 40d8072 Author: Dan HalperinAuthored: Fri Jul 1 17:21:37 2016 -0700 Committer: Dan Halperin Committed: Fri Jul 1 17:21:37 2016 -0700 -- .../examples/complete/StreamingWordExtract.java | 35 ++-- 1 file changed, 10 insertions(+), 25 deletions(-) --
[GitHub] incubator-beam pull request #576: Make examples only have optional runtime d...
Github user asfgit closed the pull request at: https://github.com/apache/incubator-beam/pull/576 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[1/2] incubator-beam git commit: Closes #576
Repository: incubator-beam Updated Branches: refs/heads/master 6887af2b1 -> 85c36b856 Closes #576 Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/85c36b85 Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/85c36b85 Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/85c36b85 Branch: refs/heads/master Commit: 85c36b856c098792c8113988e9770731c0f7da38 Parents: 6887af2 60dd7a2 Author: Dan HalperinAuthored: Fri Jul 1 17:20:17 2016 -0700 Committer: Dan Halperin Committed: Fri Jul 1 17:20:17 2016 -0700 -- examples/java/pom.xml | 1 + examples/java8/pom.xml | 2 ++ 2 files changed, 3 insertions(+) --
[2/2] incubator-beam git commit: Make examples only have optional runtime deps on runners
Make examples only have optional runtime deps on runners Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/60dd7a27 Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/60dd7a27 Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/60dd7a27 Branch: refs/heads/master Commit: 60dd7a27a11772458ad4d514cf081f2a67800572 Parents: 6887af2 Author: Kenneth KnowlesAuthored: Fri Jul 1 13:45:14 2016 -0700 Committer: Dan Halperin Committed: Fri Jul 1 17:20:17 2016 -0700 -- examples/java/pom.xml | 1 + examples/java8/pom.xml | 2 ++ 2 files changed, 3 insertions(+) -- http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/60dd7a27/examples/java/pom.xml -- diff --git a/examples/java/pom.xml b/examples/java/pom.xml index 6173ce3..0d2f505 100644 --- a/examples/java/pom.xml +++ b/examples/java/pom.xml @@ -268,6 +268,7 @@ beam-runners-direct-java ${project.version} runtime + true http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/60dd7a27/examples/java8/pom.xml -- diff --git a/examples/java8/pom.xml b/examples/java8/pom.xml index ebc6f3d..61b8cb4 100644 --- a/examples/java8/pom.xml +++ b/examples/java8/pom.xml @@ -170,6 +170,7 @@ beam-runners-direct-java ${project.version} runtime + true @@ -177,6 +178,7 @@ beam-runners-flink_2.10 ${project.version} runtime + true
[GitHub] incubator-beam pull request #579: Replace PubsubIO and injector with TextIO ...
Github user asfgit closed the pull request at: https://github.com/apache/incubator-beam/pull/579 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[1/2] incubator-beam git commit: Replace PubsubIO and injector with TextIO in traffic examples
Repository: incubator-beam Updated Branches: refs/heads/master 8254b952f -> 6887af2b1 Replace PubsubIO and injector with TextIO in traffic examples Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/e3ee1b72 Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/e3ee1b72 Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/e3ee1b72 Branch: refs/heads/master Commit: e3ee1b7279bf8d808e149be639d77a705e224a47 Parents: 8254b95 Author: Pei HeAuthored: Fri Jul 1 14:45:43 2016 -0700 Committer: Dan Halperin Committed: Fri Jul 1 17:05:52 2016 -0700 -- .../examples/complete/TrafficMaxLaneFlow.java | 64 .../beam/examples/complete/TrafficRoutes.java | 62 +++ 2 files changed, 22 insertions(+), 104 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/e3ee1b72/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java -- diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java index cfef311..2db7c9e 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java @@ -20,13 +20,11 @@ package org.apache.beam.examples.complete; import org.apache.beam.examples.common.DataflowExampleOptions; import org.apache.beam.examples.common.DataflowExampleUtils; import org.apache.beam.examples.common.ExampleBigQueryTableOptions; -import org.apache.beam.examples.common.ExamplePubsubTopicAndSubscriptionOptions; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.DefaultCoder; import org.apache.beam.sdk.io.BigQueryIO; -import org.apache.beam.sdk.io.PubsubIO; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.Description; @@ -46,7 +44,6 @@ import com.google.api.services.bigquery.model.TableFieldSchema; import com.google.api.services.bigquery.model.TableReference; import com.google.api.services.bigquery.model.TableRow; import com.google.api.services.bigquery.model.TableSchema; -import com.google.common.base.Strings; import org.apache.avro.reflect.Nullable; import org.joda.time.Duration; @@ -62,30 +59,19 @@ import java.util.List; * A Dataflow Example that runs in both batch and streaming modes with traffic sensor data. * You can configure the running mode by setting {@literal --streaming} to true or false. * - * Concepts: The batch and streaming runners, sliding windows, Google Cloud Pub/Sub - * topic injection, use of the AvroCoder to encode a custom class, and custom Combine transforms. + * Concepts: The batch and streaming runners, sliding windows, + * use of the AvroCoder to encode a custom class, and custom Combine transforms. * * This example analyzes traffic sensor data using SlidingWindows. For each window, * it finds the lane that had the highest flow recorded, for each sensor station. It writes * those max values along with auxiliary info to a BigQuery table. * - * In batch mode, the pipeline reads traffic sensor data from {@literal --inputFile}. + * The pipeline reads traffic sensor data from {@literal --inputFile}. * - * In streaming mode, the pipeline reads the data from a Pub/Sub topic. - * By default, the example will run a separate pipeline to inject the data from the default - * {@literal --inputFile} to the Pub/Sub {@literal --pubsubTopic}. It will make it available for - * the streaming pipeline to process. You may override the default {@literal --inputFile} with the - * file of your choosing. You may also set {@literal --inputFile} to an empty string, which will - * disable the automatic Pub/Sub injection, and allow you to use separate tool to control the input - * to this example. An example code, which publishes traffic sensor data to a Pub/Sub topic, - * is provided in - * https://github.com/GoogleCloudPlatform/cloud-pubsub-samples-python/tree/master/gce-cmdline-publisher;>. - * - * The example is configured to use the default Pub/Sub topic and the default BigQuery table - * from the example common package (there are no defaults for a general Dataflow pipeline). - * You can override them by using the {@literal --pubsubTopic}, {@literal --bigQueryDataset}, and - * {@literal --bigQueryTable} options. If the Pub/Sub topic or the BigQuery table do not exist, - * the example will try to create
[2/2] incubator-beam git commit: Closes #579
Closes #579 Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/6887af2b Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/6887af2b Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/6887af2b Branch: refs/heads/master Commit: 6887af2b1a92197c1fa102dd4fe2afefd3d8a23f Parents: 8254b95 e3ee1b7 Author: Dan HalperinAuthored: Fri Jul 1 17:05:53 2016 -0700 Committer: Dan Halperin Committed: Fri Jul 1 17:05:53 2016 -0700 -- .../examples/complete/TrafficMaxLaneFlow.java | 64 .../beam/examples/complete/TrafficRoutes.java | 62 +++ 2 files changed, 22 insertions(+), 104 deletions(-) --
[jira] [Commented] (BEAM-362) Move shared runner functionality out of SDK and into runners/core-java
[ https://issues.apache.org/jira/browse/BEAM-362?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15359746#comment-15359746 ] ASF GitHub Bot commented on BEAM-362: - Github user kennknowles closed the pull request at: https://github.com/apache/incubator-beam/pull/581 > Move shared runner functionality out of SDK and into runners/core-java > -- > > Key: BEAM-362 > URL: https://issues.apache.org/jira/browse/BEAM-362 > Project: Beam > Issue Type: Improvement > Components: runner-core >Reporter: Kenneth Knowles >Assignee: Kenneth Knowles > -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[GitHub] incubator-beam pull request #581: [BEAM-362] Move aggregator scraping util c...
Github user kennknowles closed the pull request at: https://github.com/apache/incubator-beam/pull/581 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[jira] [Commented] (BEAM-362) Move shared runner functionality out of SDK and into runners/core-java
[ https://issues.apache.org/jira/browse/BEAM-362?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15359713#comment-15359713 ] ASF GitHub Bot commented on BEAM-362: - Github user kennknowles closed the pull request at: https://github.com/apache/incubator-beam/pull/580 > Move shared runner functionality out of SDK and into runners/core-java > -- > > Key: BEAM-362 > URL: https://issues.apache.org/jira/browse/BEAM-362 > Project: Beam > Issue Type: Improvement > Components: runner-core >Reporter: Kenneth Knowles >Assignee: Kenneth Knowles > -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[GitHub] incubator-beam pull request #580: [BEAM-362] Move TestCountingSource to runn...
Github user kennknowles closed the pull request at: https://github.com/apache/incubator-beam/pull/580 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[jira] [Commented] (BEAM-362) Move shared runner functionality out of SDK and into runners/core-java
[ https://issues.apache.org/jira/browse/BEAM-362?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15359712#comment-15359712 ] ASF GitHub Bot commented on BEAM-362: - GitHub user kennknowles opened a pull request: https://github.com/apache/incubator-beam/pull/581 [BEAM-362] Move aggregator scraping util code to runners/core-java Be sure to do all of the following to help us incorporate your contribution quickly and easily: - [x] Make sure the PR title is formatted like: `[BEAM-] Description of pull request` - [ ] Make sure tests pass via `mvn clean verify`. (Even better, enable Travis-CI on your fork and ensure the whole test matrix passes). - [x] Replace `` in the title with the actual Jira issue number, if there is one. - [x] If this contribution is large, please file an Apache [Individual Contributor License Agreement](https://www.apache.org/licenses/icla.txt). --- You can merge this pull request into a Git repository by running: $ git pull https://github.com/kennknowles/incubator-beam runners-core-aggregators Alternatively you can review and apply these changes as the patch at: https://github.com/apache/incubator-beam/pull/581.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #581 commit 15d790698978b994b2e1d854cf4b5b20e8fd0aad Author: Kenneth KnowlesDate: 2016-07-01T21:39:03Z Move aggregator scraping util code to runners/core-java commit 7ca450df8c8bacb1d90b86dd2f5dbe074843a01f Author: Kenneth Knowles Date: 2016-07-01T21:56:20Z Move user-facing aggregator classes out of runners namespace > Move shared runner functionality out of SDK and into runners/core-java > -- > > Key: BEAM-362 > URL: https://issues.apache.org/jira/browse/BEAM-362 > Project: Beam > Issue Type: Improvement > Components: runner-core >Reporter: Kenneth Knowles >Assignee: Kenneth Knowles > -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[GitHub] incubator-beam pull request #581: [BEAM-362] Move aggregator scraping util c...
GitHub user kennknowles opened a pull request: https://github.com/apache/incubator-beam/pull/581 [BEAM-362] Move aggregator scraping util code to runners/core-java Be sure to do all of the following to help us incorporate your contribution quickly and easily: - [x] Make sure the PR title is formatted like: `[BEAM-] Description of pull request` - [ ] Make sure tests pass via `mvn clean verify`. (Even better, enable Travis-CI on your fork and ensure the whole test matrix passes). - [x] Replace `` in the title with the actual Jira issue number, if there is one. - [x] If this contribution is large, please file an Apache [Individual Contributor License Agreement](https://www.apache.org/licenses/icla.txt). --- You can merge this pull request into a Git repository by running: $ git pull https://github.com/kennknowles/incubator-beam runners-core-aggregators Alternatively you can review and apply these changes as the patch at: https://github.com/apache/incubator-beam/pull/581.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #581 commit 15d790698978b994b2e1d854cf4b5b20e8fd0aad Author: Kenneth KnowlesDate: 2016-07-01T21:39:03Z Move aggregator scraping util code to runners/core-java commit 7ca450df8c8bacb1d90b86dd2f5dbe074843a01f Author: Kenneth Knowles Date: 2016-07-01T21:56:20Z Move user-facing aggregator classes out of runners namespace --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[jira] [Commented] (BEAM-362) Move shared runner functionality out of SDK and into runners/core-java
[ https://issues.apache.org/jira/browse/BEAM-362?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15359706#comment-15359706 ] ASF GitHub Bot commented on BEAM-362: - Github user kennknowles closed the pull request at: https://github.com/apache/incubator-beam/pull/578 > Move shared runner functionality out of SDK and into runners/core-java > -- > > Key: BEAM-362 > URL: https://issues.apache.org/jira/browse/BEAM-362 > Project: Beam > Issue Type: Improvement > Components: runner-core >Reporter: Kenneth Knowles >Assignee: Kenneth Knowles > -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[GitHub] incubator-beam pull request #578: [BEAM-362] Move aggregator scraping util c...
Github user kennknowles closed the pull request at: https://github.com/apache/incubator-beam/pull/578 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[jira] [Commented] (BEAM-362) Move shared runner functionality out of SDK and into runners/core-java
[ https://issues.apache.org/jira/browse/BEAM-362?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15359705#comment-15359705 ] ASF GitHub Bot commented on BEAM-362: - GitHub user kennknowles opened a pull request: https://github.com/apache/incubator-beam/pull/580 [BEAM-362] Move TestCountingSource to runners/core-java Be sure to do all of the following to help us incorporate your contribution quickly and easily: - [x] Make sure the PR title is formatted like: `[BEAM-] Description of pull request` - [ ] Make sure tests pass via `mvn clean verify`. (Even better, enable Travis-CI on your fork and ensure the whole test matrix passes). - [x] Replace `` in the title with the actual Jira issue number, if there is one. - [x] If this contribution is large, please file an Apache [Individual Contributor License Agreement](https://www.apache.org/licenses/icla.txt). --- You can merge this pull request into a Git repository by running: $ git pull https://github.com/kennknowles/incubator-beam TestCountingSource Alternatively you can review and apply these changes as the patch at: https://github.com/apache/incubator-beam/pull/580.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #580 commit da6085a031bc2fe8d389e9993a385c73c9aaf814 Author: Kenneth KnowlesDate: 2016-07-01T21:51:21Z Move TestCountingSource to runners/core-java > Move shared runner functionality out of SDK and into runners/core-java > -- > > Key: BEAM-362 > URL: https://issues.apache.org/jira/browse/BEAM-362 > Project: Beam > Issue Type: Improvement > Components: runner-core >Reporter: Kenneth Knowles >Assignee: Kenneth Knowles > -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[GitHub] incubator-beam pull request #580: [BEAM-362] Move TestCountingSource to runn...
GitHub user kennknowles opened a pull request: https://github.com/apache/incubator-beam/pull/580 [BEAM-362] Move TestCountingSource to runners/core-java Be sure to do all of the following to help us incorporate your contribution quickly and easily: - [x] Make sure the PR title is formatted like: `[BEAM-] Description of pull request` - [ ] Make sure tests pass via `mvn clean verify`. (Even better, enable Travis-CI on your fork and ensure the whole test matrix passes). - [x] Replace `` in the title with the actual Jira issue number, if there is one. - [x] If this contribution is large, please file an Apache [Individual Contributor License Agreement](https://www.apache.org/licenses/icla.txt). --- You can merge this pull request into a Git repository by running: $ git pull https://github.com/kennknowles/incubator-beam TestCountingSource Alternatively you can review and apply these changes as the patch at: https://github.com/apache/incubator-beam/pull/580.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #580 commit da6085a031bc2fe8d389e9993a385c73c9aaf814 Author: Kenneth KnowlesDate: 2016-07-01T21:51:21Z Move TestCountingSource to runners/core-java --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] incubator-beam pull request #579: Replace PubsubIO and injector with TextIO ...
GitHub user peihe opened a pull request: https://github.com/apache/incubator-beam/pull/579 Replace PubsubIO and injector with TextIO in traffic examples You can merge this pull request into a Git repository by running: $ git pull https://github.com/peihe/incubator-beam streaming-examples-4 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/incubator-beam/pull/579.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #579 commit 46b21045185aaf88d95a93d72b42ea37cfd088da Author: Pei HeDate: 2016-07-01T21:45:43Z Replace PubsubIO and injector with TextIO in traffic examples --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[jira] [Commented] (BEAM-362) Move shared runner functionality out of SDK and into runners/core-java
[ https://issues.apache.org/jira/browse/BEAM-362?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15359697#comment-15359697 ] ASF GitHub Bot commented on BEAM-362: - GitHub user kennknowles opened a pull request: https://github.com/apache/incubator-beam/pull/578 [BEAM-362] Move aggregator scraping util code to runners/core-java Be sure to do all of the following to help us incorporate your contribution quickly and easily: - [x] Make sure the PR title is formatted like: `[BEAM-] Description of pull request` - [ ] Make sure tests pass via `mvn clean verify`. (Even better, enable Travis-CI on your fork and ensure the whole test matrix passes). - [x] Replace `` in the title with the actual Jira issue number, if there is one. - [x] If this contribution is large, please file an Apache [Individual Contributor License Agreement](https://www.apache.org/licenses/icla.txt). --- You can merge this pull request into a Git repository by running: $ git pull https://github.com/kennknowles/incubator-beam runners-core-aggregators Alternatively you can review and apply these changes as the patch at: https://github.com/apache/incubator-beam/pull/578.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #578 commit 99947319a39eeb2446388e4520f8e67d5c7ffda4 Author: Kenneth KnowlesDate: 2016-07-01T21:39:03Z Move aggregator scraping util code to runners/core-java > Move shared runner functionality out of SDK and into runners/core-java > -- > > Key: BEAM-362 > URL: https://issues.apache.org/jira/browse/BEAM-362 > Project: Beam > Issue Type: Improvement > Components: runner-core >Reporter: Kenneth Knowles >Assignee: Kenneth Knowles > -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[GitHub] incubator-beam pull request #578: [BEAM-362] Move aggregator scraping util c...
GitHub user kennknowles opened a pull request: https://github.com/apache/incubator-beam/pull/578 [BEAM-362] Move aggregator scraping util code to runners/core-java Be sure to do all of the following to help us incorporate your contribution quickly and easily: - [x] Make sure the PR title is formatted like: `[BEAM-] Description of pull request` - [ ] Make sure tests pass via `mvn clean verify`. (Even better, enable Travis-CI on your fork and ensure the whole test matrix passes). - [x] Replace `` in the title with the actual Jira issue number, if there is one. - [x] If this contribution is large, please file an Apache [Individual Contributor License Agreement](https://www.apache.org/licenses/icla.txt). --- You can merge this pull request into a Git repository by running: $ git pull https://github.com/kennknowles/incubator-beam runners-core-aggregators Alternatively you can review and apply these changes as the patch at: https://github.com/apache/incubator-beam/pull/578.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #578 commit 99947319a39eeb2446388e4520f8e67d5c7ffda4 Author: Kenneth KnowlesDate: 2016-07-01T21:39:03Z Move aggregator scraping util code to runners/core-java --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] incubator-beam pull request #577: Job Messages Prototypes
GitHub user swegner opened a pull request: https://github.com/apache/incubator-beam/pull/577 Job Messages Prototypes [not to be submitted, just for prototyping feedback] You can merge this pull request into a Git repository by running: $ git pull https://github.com/swegner/incubator-beam dyndispdata-prototypes Alternatively you can review and apply these changes as the patch at: https://github.com/apache/incubator-beam/pull/577.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #577 commit 4394ecbab2abfb061681c0484bdff05338085b0d Author: Scott WegnerDate: 2016-07-01T21:40:02Z Initial prototype example usage --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[jira] [Commented] (BEAM-145) OutputTimeFn#assignOutputTime overrides WindowFn#getOutputTime in unfortunate ways
[ https://issues.apache.org/jira/browse/BEAM-145?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15359666#comment-15359666 ] ASF GitHub Bot commented on BEAM-145: - Github user asfgit closed the pull request at: https://github.com/apache/incubator-beam/pull/573 > OutputTimeFn#assignOutputTime overrides WindowFn#getOutputTime in unfortunate > ways > -- > > Key: BEAM-145 > URL: https://issues.apache.org/jira/browse/BEAM-145 > Project: Beam > Issue Type: Bug > Components: runner-core >Reporter: Kenneth Knowles >Assignee: Kenneth Knowles >Priority: Minor > Labels: windowing > Fix For: 0.1.0-incubating > > > Today the {{OutputTimeFn}} includes {{#assignOutputTime}}, {{#combine}}, and > {{#merge}}. Together these express the grouping of timestamps, analogous to > the grouping of values in a GBK / Combine, in a canonical way. > The default {{OutputTimeFn}} is provided by the {{WindowFn}}. In particular, > {{SlidingWindows}} provides an {{OutputTimeFn}} that shifts input timestamps > later to avoid watermark stuckness and then takes the minimum to compute the > output timestamp. > The SDK additionally provides instance for "min", "max" and "end of window" > output timestamps. > Unfortunately, if one overrides the {{OutputTimeFn}} to one of these, the > shifting done by {{SlidingWindows}} is lost. > This is actually only a minor problem for now, since "min" is the default, > "end of window" is unaffected, and "max" has only esoteric uses.The fix is > easy: > This is interrelated with another suggested change: Since there are only > three common {{OutputTimeFn}} instances, and it is a high bandwidth API, it > does not seem worthwhile to leave it in userland. So it is proposed to reduce > it to an enum, which would leave only the {{WindowFn}} as a userland place > for timestamp adjustments. (requiring special casing for end-of-window, since > it cannot be implemented without owning {{#assignOutputTime}}) -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[1/2] incubator-beam git commit: Fix timestamps in GroupAlsoByWindowsProperties
Repository: incubator-beam Updated Branches: refs/heads/master b4c123cd4 -> 8254b952f Fix timestamps in GroupAlsoByWindowsProperties Some of the timestamps were not adjusted when BEAM-145 was fixed to respect the WindowFn's timestamps. Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/190740f7 Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/190740f7 Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/190740f7 Branch: refs/heads/master Commit: 190740f7971be506df5416dc4dfc5714bba0c972 Parents: b4c123c Author: Kenneth KnowlesAuthored: Fri Jul 1 09:53:37 2016 -0700 Committer: Kenneth Knowles Committed: Fri Jul 1 09:53:37 2016 -0700 -- .../apache/beam/sdk/util/GroupAlsoByWindowsProperties.java | 8 ++-- 1 file changed, 6 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/190740f7/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsProperties.java -- diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsProperties.java b/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsProperties.java index f653f49..fe2a495 100644 --- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsProperties.java +++ b/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsProperties.java @@ -165,11 +165,13 @@ public class GroupAlsoByWindowsProperties { TimestampedValue > item1 = Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 20))); assertThat(item1.getValue().getValue(), containsInAnyOrder("v1", "v2")); +// Timestamp adjusted by WindowFn to exceed the end of the prior sliding window assertThat(item1.getTimestamp(), equalTo(new Instant(10))); TimestampedValue > item2 = Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(10, 30))); assertThat(item2.getValue().getValue(), contains("v2")); +// Timestamp adjusted by WindowFn to exceed the end of the prior sliding window assertThat(item2.getTimestamp(), equalTo(new Instant(20))); } @@ -218,13 +220,15 @@ public class GroupAlsoByWindowsProperties { Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(0, 20))); assertThat(item1.getValue().getKey(), equalTo("k")); assertThat(item1.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(1L, 2L, 4L; -assertThat(item1.getTimestamp(), equalTo(new Instant(5L))); +// Timestamp adjusted by WindowFn to exceed the end of the prior sliding window +assertThat(item1.getTimestamp(), equalTo(new Instant(10L))); TimestampedValue > item2 = Iterables.getOnlyElement(result.peekOutputElementsInWindow(window(10, 30))); assertThat(item2.getValue().getKey(), equalTo("k")); assertThat(item2.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(2L, 4L; -assertThat(item2.getTimestamp(), equalTo(new Instant(15L))); +// Timestamp adjusted by WindowFn to exceed the end of the prior sliding window +assertThat(item2.getTimestamp(), equalTo(new Instant(20L))); } /**
[GitHub] incubator-beam pull request #573: [BEAM-145] Fix timestamps in GroupAlsoByWi...
Github user asfgit closed the pull request at: https://github.com/apache/incubator-beam/pull/573 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[2/2] incubator-beam git commit: This closes #573
This closes #573 Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/8254b952 Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/8254b952 Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/8254b952 Branch: refs/heads/master Commit: 8254b952fe26d1d3467e44fca11d1f4760162a60 Parents: b4c123c 190740f Author: Kenneth KnowlesAuthored: Fri Jul 1 14:29:04 2016 -0700 Committer: Kenneth Knowles Committed: Fri Jul 1 14:29:04 2016 -0700 -- .../apache/beam/sdk/util/GroupAlsoByWindowsProperties.java | 8 ++-- 1 file changed, 6 insertions(+), 2 deletions(-) --
[jira] [Commented] (BEAM-416) Jenkins Python Verify post commit tests are timing out
[ https://issues.apache.org/jira/browse/BEAM-416?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15359628#comment-15359628 ] ASF GitHub Bot commented on BEAM-416: - Github user asfgit closed the pull request at: https://github.com/apache/incubator-beam/pull/575 > Jenkins Python Verify post commit tests are timing out > -- > > Key: BEAM-416 > URL: https://issues.apache.org/jira/browse/BEAM-416 > Project: Beam > Issue Type: Bug > Components: sdk-py >Reporter: Ahmet Altay >Assignee: Silviu Calinoiu > Attachments: job_output (2) > > > beam_PostCommit_PythonVerify is timing out at the e2e testing phase: > Console output: > https://builds.apache.org/view/Beam/job/beam_PostCommit_PythonVerify/8/console > e2e test: > https://pantheon.corp.google.com/dataflow/job/2016-07-01_08_02_45-15435546446836030984?project=apache-beam-testing > Workers are failing to find the correct container image (from worker logs): > Error syncing pod 6d3e3a71409d65aa43494143d705455b, skipping: failed to > "StartContainer" for "python" with ImagePullBackOff: "Back-off pulling image > \"dataflow.gcr.io/v1beta3/python:latest\"" > It might be related to this commit: > https://github.com/apache/incubator-beam/commit/0bda677d47d5bd5d9c45b74e00e5c3fd113a4f81 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[GitHub] incubator-beam pull request #575: [BEAM-416] Use the beamhead label for cont...
Github user asfgit closed the pull request at: https://github.com/apache/incubator-beam/pull/575 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[1/2] incubator-beam git commit: Use the beamhead label for containers
Repository: incubator-beam Updated Branches: refs/heads/python-sdk 5ab5567ea -> 253497655 Use the beamhead label for containers Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/05f45c8b Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/05f45c8b Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/05f45c8b Branch: refs/heads/python-sdk Commit: 05f45c8b268f90e74401cabe135d6a6e7b353102 Parents: 5ab5567 Author: Silviu CalinoiuAuthored: Fri Jul 1 12:59:35 2016 -0700 Committer: Silviu Calinoiu Committed: Fri Jul 1 12:59:35 2016 -0700 -- sdks/python/apache_beam/utils/dependency.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/05f45c8b/sdks/python/apache_beam/utils/dependency.py -- diff --git a/sdks/python/apache_beam/utils/dependency.py b/sdks/python/apache_beam/utils/dependency.py index 7d9bd10..ddb640a 100644 --- a/sdks/python/apache_beam/utils/dependency.py +++ b/sdks/python/apache_beam/utils/dependency.py @@ -432,8 +432,8 @@ def get_required_container_version(): return '%s.%s.%s' % pkg.parse_version(version)._version.release except pkg.DistributionNotFound: # This case covers Apache Beam end-to-end testing scenarios. All these tests -# will run with the latest container version. -return 'latest' +# will run with a special container version. +return 'beamhead' def _download_pypi_sdk_package(temp_dir):
[2/2] incubator-beam git commit: Closes #575
Closes #575 Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/25349765 Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/25349765 Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/25349765 Branch: refs/heads/python-sdk Commit: 253497655a73a64fa46e2238e7e8b1bdff1671dc Parents: 5ab5567 05f45c8 Author: Robert BradshawAuthored: Fri Jul 1 14:05:17 2016 -0700 Committer: Robert Bradshaw Committed: Fri Jul 1 14:05:17 2016 -0700 -- sdks/python/apache_beam/utils/dependency.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) --
[GitHub] incubator-beam pull request #576: Make examples only have optional runtime d...
GitHub user kennknowles opened a pull request: https://github.com/apache/incubator-beam/pull/576 Make examples only have optional runtime deps on runners Be sure to do all of the following to help us incorporate your contribution quickly and easily: - [x] Make sure the PR title is formatted like: `[BEAM-] Description of pull request` - [ ] Make sure tests pass via `mvn clean verify`. (Even better, enable Travis-CI on your fork and ensure the whole test matrix passes). - [x] Replace `` in the title with the actual Jira issue number, if there is one. - [x] If this contribution is large, please file an Apache [Individual Contributor License Agreement](https://www.apache.org/licenses/icla.txt). --- When a project depends on the `examples` modules to do some integration testing, it shouldn't pull in any particular runner. You can merge this pull request into a Git repository by running: $ git pull https://github.com/kennknowles/incubator-beam example-runners Alternatively you can review and apply these changes as the patch at: https://github.com/apache/incubator-beam/pull/576.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #576 commit 2cfd4c7f55fc58536a0aee83d4c09ee857931343 Author: Kenneth KnowlesDate: 2016-07-01T20:45:14Z Make examples only have optional runtime deps on runners --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[jira] [Commented] (BEAM-416) Jenkins Python Verify post commit tests are timing out
[ https://issues.apache.org/jira/browse/BEAM-416?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15359548#comment-15359548 ] ASF GitHub Bot commented on BEAM-416: - GitHub user silviulica opened a pull request: https://github.com/apache/incubator-beam/pull/575 [BEAM-416] Use the beamhead label for containers Use a special container label for Beam SDKs running E2E tests. You can merge this pull request into a Git repository by running: $ git pull https://github.com/silviulica/incubator-beam beamhead Alternatively you can review and apply these changes as the patch at: https://github.com/apache/incubator-beam/pull/575.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #575 commit 05f45c8b268f90e74401cabe135d6a6e7b353102 Author: Silviu CalinoiuDate: 2016-07-01T19:59:35Z Use the beamhead label for containers > Jenkins Python Verify post commit tests are timing out > -- > > Key: BEAM-416 > URL: https://issues.apache.org/jira/browse/BEAM-416 > Project: Beam > Issue Type: Bug > Components: sdk-py >Reporter: Ahmet Altay >Assignee: Silviu Calinoiu > Attachments: job_output (2) > > > beam_PostCommit_PythonVerify is timing out at the e2e testing phase: > Console output: > https://builds.apache.org/view/Beam/job/beam_PostCommit_PythonVerify/8/console > e2e test: > https://pantheon.corp.google.com/dataflow/job/2016-07-01_08_02_45-15435546446836030984?project=apache-beam-testing > Workers are failing to find the correct container image (from worker logs): > Error syncing pod 6d3e3a71409d65aa43494143d705455b, skipping: failed to > "StartContainer" for "python" with ImagePullBackOff: "Back-off pulling image > \"dataflow.gcr.io/v1beta3/python:latest\"" > It might be related to this commit: > https://github.com/apache/incubator-beam/commit/0bda677d47d5bd5d9c45b74e00e5c3fd113a4f81 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[GitHub] incubator-beam pull request #574: Fix, suppress, or file bugs for FindBugs i...
GitHub user swegner opened a pull request: https://github.com/apache/incubator-beam/pull/574 Fix, suppress, or file bugs for FindBugs issues Be sure to do all of the following to help us incorporate your contribution quickly and easily: - [ ] Make sure the PR title is formatted like: `[BEAM-] Description of pull request` - [ ] Make sure tests pass via `mvn clean verify`. (Even better, enable Travis-CI on your fork and ensure the whole test matrix passes). - [ ] Replace `` in the title with the actual Jira issue number, if there is one. - [ ] If this contribution is large, please file an Apache [Individual Contributor License Agreement](https://www.apache.org/licenses/icla.txt). --- When adding FindBugs integration, many existing issues were uncovered. Of those issues, some are trivial to fix, some are more complicated, and some are unimportant or false positive. This PR goes through and triages the existing baseline issues. For each: * Unimportant / false positive: Move to inline suppression with justification ([example](https://github.com/apache/incubator-beam/compare/master...swegner:findbugs-stale?expand=1#diff-c1df753c9b67cc72f7beed9eaa90R64)) * Trivial fix: Fixed inline ([example](https://github.com/apache/incubator-beam/compare/master...swegner:findbugs-stale?expand=1#diff-d6c918ab6169a2f61f4902c6018778f3R351)) * Non-trivial fix needed: Opened bug and tagged baseline suppression ([example](https://github.com/apache/incubator-beam/compare/master...swegner:findbugs-stale?expand=1#diff-6bffc9239f6c94cf1540559353cd868dR36)) You can merge this pull request into a Git repository by running: $ git pull https://github.com/swegner/incubator-beam findbugs-stale Alternatively you can review and apply these changes as the patch at: https://github.com/apache/incubator-beam/pull/574.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #574 commit c542117c48ce7983b363d676220392cd12b88ed7 Author: Scott WegnerDate: 2016-06-30T23:16:14Z Fix and file bugs for FindBugs issues --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[jira] [Updated] (BEAM-421) StateTags$CombiningValueStateTag inherits equals from super class but without incorporating its state
[ https://issues.apache.org/jira/browse/BEAM-421?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Scott Wegner updated BEAM-421: -- Description: [FindBugs EQ_DOESNT_OVERRIDE_EQUALS|https://github.com/apache/incubator-beam/blob/58a029a06aea1030279e5da8f9fa3114f456c1db/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml#L555]: Class doesn't override equals in superclass Applies to: [StateTags$CombiningValueStateTag|https://github.com/apache/incubator-beam/blob/58a029a06aea1030279e5da8f9fa3114f456c1db/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTags.java#L344]. CombiningValueStateTag inherits its equals implementation from its parent class, KeyedCombiningValueStateTag. CombiningValueStateTag has additional fields which are not included in the equality implementation, which may be incorrect. This is a good starter bug. When fixing, please remove the corresponding entries from [findbugs-filter.xml|https://github.com/apache/incubator-beam/blob/master/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml] and verify the build passes. was: [FindBugs SE_BAD_FIELD|https://github.com/apache/incubator-beam/blob/58a029a06aea1030279e5da8f9fa3114f456c1db/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml#L477]: Non-transient non-serializable instance field in serializable class Applies to: [WatermarkHold.timerInternals|https://github.com/apache/incubator-beam/blob/58a029a06aea1030279e5da8f9fa3114f456c1db/runners/core-java/src/main/java/org/apache/beam/sdk/util/WatermarkHold.java#L81]. This is a good starter bug. When fixing, please remove the corresponding entries from [findbugs-filter.xml|https://github.com/apache/incubator-beam/blob/master/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml] and verify the build passes. > StateTags$CombiningValueStateTag inherits equals from super class but without > incorporating its state > - > > Key: BEAM-421 > URL: https://issues.apache.org/jira/browse/BEAM-421 > Project: Beam > Issue Type: Bug > Components: sdk-java-core >Reporter: Scott Wegner >Priority: Minor > Labels: findbugs, newbie, starter > > [FindBugs > EQ_DOESNT_OVERRIDE_EQUALS|https://github.com/apache/incubator-beam/blob/58a029a06aea1030279e5da8f9fa3114f456c1db/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml#L555]: > Class doesn't override equals in superclass > Applies to: > [StateTags$CombiningValueStateTag|https://github.com/apache/incubator-beam/blob/58a029a06aea1030279e5da8f9fa3114f456c1db/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTags.java#L344]. > CombiningValueStateTag inherits its equals implementation from its parent > class, KeyedCombiningValueStateTag. CombiningValueStateTag has additional > fields which are not included in the equality implementation, which may be > incorrect. > This is a good starter bug. When fixing, please remove the corresponding > entries from > [findbugs-filter.xml|https://github.com/apache/incubator-beam/blob/master/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml] > and verify the build passes. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (BEAM-421) StateTags$CombiningValueStateTag inherits equals from super class but without incorporating its state
Scott Wegner created BEAM-421: - Summary: StateTags$CombiningValueStateTag inherits equals from super class but without incorporating its state Key: BEAM-421 URL: https://issues.apache.org/jira/browse/BEAM-421 Project: Beam Issue Type: Bug Components: sdk-java-core Reporter: Scott Wegner Priority: Minor [FindBugs SE_BAD_FIELD|https://github.com/apache/incubator-beam/blob/58a029a06aea1030279e5da8f9fa3114f456c1db/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml#L477]: Non-transient non-serializable instance field in serializable class Applies to: [WatermarkHold.timerInternals|https://github.com/apache/incubator-beam/blob/58a029a06aea1030279e5da8f9fa3114f456c1db/runners/core-java/src/main/java/org/apache/beam/sdk/util/WatermarkHold.java#L81]. This is a good starter bug. When fixing, please remove the corresponding entries from [findbugs-filter.xml|https://github.com/apache/incubator-beam/blob/master/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml] and verify the build passes. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (BEAM-420) Non-transient non-serializable instance field in WatermarkHold
[ https://issues.apache.org/jira/browse/BEAM-420?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Scott Wegner updated BEAM-420: -- Description: [FindBugs SE_BAD_FIELD|https://github.com/apache/incubator-beam/blob/58a029a06aea1030279e5da8f9fa3114f456c1db/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml#L477]: Non-transient non-serializable instance field in serializable class Applies to: [WatermarkHold.timerInternals|https://github.com/apache/incubator-beam/blob/58a029a06aea1030279e5da8f9fa3114f456c1db/runners/core-java/src/main/java/org/apache/beam/sdk/util/WatermarkHold.java#L81]. This is a good starter bug. When fixing, please remove the corresponding entries from [findbugs-filter.xml|https://github.com/apache/incubator-beam/blob/master/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml] and verify the build passes. was: [FindBugs SE_BAD_FIELD|https://github.com/apache/incubator-beam/blob/58a029a06aea1030279e5da8f9fa3114f456c1db/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml#L363]: Non-transient non-serializable instance field in serializable class Applies to: [CombineFnUtil$NonSerializableBoundedKeyedCombineFn.context|https://github.com/apache/incubator-beam/blob/58a029a06aea1030279e5da8f9fa3114f456c1db/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineFnUtil.java#L170]. This is a good starter bug. When fixing, please remove the corresponding entries from [findbugs-filter.xml|https://github.com/apache/incubator-beam/blob/master/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml] and verify the build passes. > Non-transient non-serializable instance field in WatermarkHold > -- > > Key: BEAM-420 > URL: https://issues.apache.org/jira/browse/BEAM-420 > Project: Beam > Issue Type: Bug > Components: sdk-java-core >Reporter: Scott Wegner >Priority: Minor > Labels: findbugs, newbie, starter > > [FindBugs > SE_BAD_FIELD|https://github.com/apache/incubator-beam/blob/58a029a06aea1030279e5da8f9fa3114f456c1db/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml#L477]: > Non-transient non-serializable instance field in serializable class > Applies to: > [WatermarkHold.timerInternals|https://github.com/apache/incubator-beam/blob/58a029a06aea1030279e5da8f9fa3114f456c1db/runners/core-java/src/main/java/org/apache/beam/sdk/util/WatermarkHold.java#L81]. > This is a good starter bug. When fixing, please remove the corresponding > entries from > [findbugs-filter.xml|https://github.com/apache/incubator-beam/blob/master/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml] > and verify the build passes. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (BEAM-420) Non-transient non-serializable instance field in WatermarkHold
Scott Wegner created BEAM-420: - Summary: Non-transient non-serializable instance field in WatermarkHold Key: BEAM-420 URL: https://issues.apache.org/jira/browse/BEAM-420 Project: Beam Issue Type: Bug Components: sdk-java-core Reporter: Scott Wegner Priority: Minor [FindBugs SE_BAD_FIELD|https://github.com/apache/incubator-beam/blob/58a029a06aea1030279e5da8f9fa3114f456c1db/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml#L363]: Non-transient non-serializable instance field in serializable class Applies to: [CombineFnUtil$NonSerializableBoundedKeyedCombineFn.context|https://github.com/apache/incubator-beam/blob/58a029a06aea1030279e5da8f9fa3114f456c1db/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineFnUtil.java#L170]. This is a good starter bug. When fixing, please remove the corresponding entries from [findbugs-filter.xml|https://github.com/apache/incubator-beam/blob/master/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml] and verify the build passes. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Closed] (BEAM-417) AvroUtils$AvroMetadata exposes mutable syncMarker
[ https://issues.apache.org/jira/browse/BEAM-417?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Scott Wegner closed BEAM-417. - Resolution: Invalid Assignee: Scott Wegner Fix Version/s: Not applicable > AvroUtils$AvroMetadata exposes mutable syncMarker > - > > Key: BEAM-417 > URL: https://issues.apache.org/jira/browse/BEAM-417 > Project: Beam > Issue Type: Bug > Components: sdk-java-core >Reporter: Scott Wegner >Assignee: Scott Wegner >Priority: Minor > Labels: findbugs, newbie, starter > Fix For: Not applicable > > > [FindBugs > EI_EXPOSE_REP|https://github.com/apache/incubator-beam/blob/58a029a06aea1030279e5da8f9fa3114f456c1db/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml#L351]: > May expose internal representation by returning reference to mutable object > Applies to: > [AvroUtils$AvroMetadata.getSyncMarker|https://github.com/apache/incubator-beam/blob/58a029a06aea1030279e5da8f9fa3114f456c1db/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AvroUtils.java#L92]. > This is a good starter bug. When fixing, please remove the corresponding > entries from > [findbugs-filter.xml|https://github.com/apache/incubator-beam/blob/master/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml] > and verify the build passes. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[GitHub] incubator-beam pull request #570: Enables unused-import and used-before-assi...
Github user asfgit closed the pull request at: https://github.com/apache/incubator-beam/pull/570 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[2/3] incubator-beam git commit: Set end value in window.py and remove pylint disable statement.
Set end value in window.py and remove pylint disable statement. Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/833a4b6d Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/833a4b6d Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/833a4b6d Branch: refs/heads/python-sdk Commit: 833a4b6de34186b976a1ad6b0d6894dc3044a371 Parents: 2094e00 Author: Ahmet AltayAuthored: Fri Jul 1 10:28:38 2016 -0700 Committer: Ahmet Altay Committed: Fri Jul 1 10:28:38 2016 -0700 -- sdks/python/apache_beam/transforms/window.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/833a4b6d/sdks/python/apache_beam/transforms/window.py -- diff --git a/sdks/python/apache_beam/transforms/window.py b/sdks/python/apache_beam/transforms/window.py index cc17bcf..eec89c0 100644 --- a/sdks/python/apache_beam/transforms/window.py +++ b/sdks/python/apache_beam/transforms/window.py @@ -367,9 +367,10 @@ class Sessions(WindowFn): def merge(self, merge_context): to_merge = [] +end = timeutil.MIN_TIMESTAMP for w in sorted(merge_context.windows, key=lambda w: w.start): if to_merge: -if end > w.start: # pylint: disable=used-before-assignment +if end > w.start: to_merge.append(w) if w.end > end: end = w.end
[3/3] incubator-beam git commit: This closes #570
This closes #570 Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/5ab5567e Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/5ab5567e Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/5ab5567e Branch: refs/heads/python-sdk Commit: 5ab5567ea68431531eea0543e5c046d5936d4268 Parents: df7ecd5 833a4b6 Author: Kenneth KnowlesAuthored: Fri Jul 1 10:50:31 2016 -0700 Committer: Kenneth Knowles Committed: Fri Jul 1 10:50:31 2016 -0700 -- sdks/python/.pylintrc | 2 -- sdks/python/apache_beam/coders/coder_impl.py| 1 - .../apache_beam/examples/complete/autocomplete.py | 2 +- .../examples/complete/autocomplete_test.py | 6 +- .../apache_beam/examples/complete/estimate_pi.py| 4 ++-- sdks/python/apache_beam/examples/complete/tfidf.py | 6 +++--- .../examples/complete/top_wikipedia_sessions.py | 9 + .../examples/cookbook/bigquery_side_input.py| 6 +++--- .../examples/cookbook/bigquery_side_input_test.py | 10 +- .../examples/cookbook/bigquery_tornadoes_test.py| 2 +- sdks/python/apache_beam/examples/cookbook/coders.py | 11 +-- .../python/apache_beam/examples/cookbook/filters.py | 4 ++-- .../apache_beam/examples/cookbook/filters_test.py | 2 +- .../apache_beam/examples/cookbook/mergecontacts.py | 16 .../examples/cookbook/multiple_output_pardo.py | 8 +--- sdks/python/apache_beam/internal/json_value_test.py | 1 - sdks/python/apache_beam/transforms/core.py | 3 --- sdks/python/apache_beam/transforms/ptransform.py| 1 - sdks/python/apache_beam/transforms/util.py | 3 --- sdks/python/apache_beam/transforms/window.py| 1 + 20 files changed, 43 insertions(+), 55 deletions(-) --
[jira] [Closed] (BEAM-402) StructuralByteArray exposes mutable internal byte array
[ https://issues.apache.org/jira/browse/BEAM-402?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Scott Wegner closed BEAM-402. - Resolution: Invalid Assignee: Scott Wegner Fix Version/s: Not applicable > StructuralByteArray exposes mutable internal byte array > --- > > Key: BEAM-402 > URL: https://issues.apache.org/jira/browse/BEAM-402 > Project: Beam > Issue Type: Bug > Components: sdk-java-core >Reporter: Scott Wegner >Assignee: Scott Wegner >Priority: Minor > Labels: findbugs, newbie, starter > Fix For: Not applicable > > > [FindBugs > EI_EXPOSE_REP|https://github.com/apache/incubator-beam/blob/58a029a06aea1030279e5da8f9fa3114f456c1db/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml#L112]: > May expose internal representation by returning reference to mutable object > Applies to: > [StructuralByteArray|https://github.com/apache/incubator-beam/blob/58a029a06aea1030279e5da8f9fa3114f456c1db/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StructuralByteArray.java]. > Based on usage, it's not entirely clear whether this needs to be fixed. If > not, the FindBugs suppression should be moved out of > [findbugs-filter.xml|https://github.com/apache/incubator-beam/blob/58a029a06aea1030279e5da8f9fa3114f456c1db/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml#L112] > and inline as a @SuppressFBWarnings annotation. > This is a good starter bug. When fixing, please remove the corresponding > entries from > [findbugs-filter.xml|https://github.com/apache/incubator-beam/blob/master/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml] > and verify the build passes. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (BEAM-419) Non-transient non-serializable instance field in CombineFnUtil$NonSerializableBoundedKeyedCombineFn
[ https://issues.apache.org/jira/browse/BEAM-419?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Scott Wegner updated BEAM-419: -- Description: [FindBugs SE_BAD_FIELD|https://github.com/apache/incubator-beam/blob/58a029a06aea1030279e5da8f9fa3114f456c1db/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml#L363]: Non-transient non-serializable instance field in serializable class Applies to: [CombineFnUtil$NonSerializableBoundedKeyedCombineFn.context|https://github.com/apache/incubator-beam/blob/58a029a06aea1030279e5da8f9fa3114f456c1db/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineFnUtil.java#L170]. This is a good starter bug. When fixing, please remove the corresponding entries from [findbugs-filter.xml|https://github.com/apache/incubator-beam/blob/master/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml] and verify the build passes. was: [FindBugs SE_TRANSIENT_FIELD_NOT_RESTORED|https://github.com/apache/incubator-beam/blob/58a029a06aea1030279e5da8f9fa3114f456c1db/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml#L357]: Transient field that isn't set by deserialization. Applies to: [BitSetCoder.byteArrayCoder|https://github.com/apache/incubator-beam/blob/58a029a06aea1030279e5da8f9fa3114f456c1db/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BitSetCoder.java#L35]. This is a good starter bug. When fixing, please remove the corresponding entries from [findbugs-filter.xml|https://github.com/apache/incubator-beam/blob/master/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml] and verify the build passes. > Non-transient non-serializable instance field in > CombineFnUtil$NonSerializableBoundedKeyedCombineFn > --- > > Key: BEAM-419 > URL: https://issues.apache.org/jira/browse/BEAM-419 > Project: Beam > Issue Type: Bug > Components: sdk-java-core >Reporter: Scott Wegner >Priority: Minor > Labels: findbugs, newbie, starter > > [FindBugs > SE_BAD_FIELD|https://github.com/apache/incubator-beam/blob/58a029a06aea1030279e5da8f9fa3114f456c1db/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml#L363]: > Non-transient non-serializable instance field in serializable class > Applies to: > [CombineFnUtil$NonSerializableBoundedKeyedCombineFn.context|https://github.com/apache/incubator-beam/blob/58a029a06aea1030279e5da8f9fa3114f456c1db/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineFnUtil.java#L170]. > This is a good starter bug. When fixing, please remove the corresponding > entries from > [findbugs-filter.xml|https://github.com/apache/incubator-beam/blob/master/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml] > and verify the build passes. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (BEAM-419) Non-transient non-serializable instance field in CombineFnUtil$NonSerializableBoundedKeyedCombineFn
Scott Wegner created BEAM-419: - Summary: Non-transient non-serializable instance field in CombineFnUtil$NonSerializableBoundedKeyedCombineFn Key: BEAM-419 URL: https://issues.apache.org/jira/browse/BEAM-419 Project: Beam Issue Type: Bug Components: sdk-java-core Reporter: Scott Wegner Priority: Minor [FindBugs SE_TRANSIENT_FIELD_NOT_RESTORED|https://github.com/apache/incubator-beam/blob/58a029a06aea1030279e5da8f9fa3114f456c1db/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml#L357]: Transient field that isn't set by deserialization. Applies to: [BitSetCoder.byteArrayCoder|https://github.com/apache/incubator-beam/blob/58a029a06aea1030279e5da8f9fa3114f456c1db/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BitSetCoder.java#L35]. This is a good starter bug. When fixing, please remove the corresponding entries from [findbugs-filter.xml|https://github.com/apache/incubator-beam/blob/master/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml] and verify the build passes. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (BEAM-417) AvroUtils$AvroMetadata exposes mutable syncMarker
Scott Wegner created BEAM-417: - Summary: AvroUtils$AvroMetadata exposes mutable syncMarker Key: BEAM-417 URL: https://issues.apache.org/jira/browse/BEAM-417 Project: Beam Issue Type: Bug Components: sdk-java-core Reporter: Scott Wegner Priority: Minor [FindBugs FE_FLOATING_POINT_EQUALITY|https://github.com/apache/incubator-beam/blob/58a029a06aea1030279e5da8f9fa3114f456c1db/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml#L298]: Test for floating point equality Applies to: [Mean$CountSum.equals|https://github.com/apache/incubator-beam/blob/58a029a06aea1030279e5da8f9fa3114f456c1db/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Mean.java#L165]. This is a good starter bug. When fixing, please remove the corresponding entries from [findbugs-filter.xml|https://github.com/apache/incubator-beam/blob/master/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml] and verify the build passes. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (BEAM-418) BitSetCoder transient field byteArrayCoder not set after deserialization
Scott Wegner created BEAM-418: - Summary: BitSetCoder transient field byteArrayCoder not set after deserialization Key: BEAM-418 URL: https://issues.apache.org/jira/browse/BEAM-418 Project: Beam Issue Type: Bug Components: sdk-java-core Reporter: Scott Wegner Priority: Minor [FindBugs EI_EXPOSE_REP|https://github.com/apache/incubator-beam/blob/58a029a06aea1030279e5da8f9fa3114f456c1db/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml#L351]: May expose internal representation by returning reference to mutable object Applies to: [AvroUtils$AvroMetadata.getSyncMarker|https://github.com/apache/incubator-beam/blob/58a029a06aea1030279e5da8f9fa3114f456c1db/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AvroUtils.java#L92]. This is a good starter bug. When fixing, please remove the corresponding entries from [findbugs-filter.xml|https://github.com/apache/incubator-beam/blob/master/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml] and verify the build passes. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (BEAM-418) BitSetCoder transient field byteArrayCoder not set after deserialization
[ https://issues.apache.org/jira/browse/BEAM-418?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Scott Wegner updated BEAM-418: -- Description: [FindBugs SE_TRANSIENT_FIELD_NOT_RESTORED|https://github.com/apache/incubator-beam/blob/58a029a06aea1030279e5da8f9fa3114f456c1db/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml#L357]: Transient field that isn't set by deserialization. Applies to: [BitSetCoder.byteArrayCoder|https://github.com/apache/incubator-beam/blob/58a029a06aea1030279e5da8f9fa3114f456c1db/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BitSetCoder.java#L35]. This is a good starter bug. When fixing, please remove the corresponding entries from [findbugs-filter.xml|https://github.com/apache/incubator-beam/blob/master/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml] and verify the build passes. was: [FindBugs EI_EXPOSE_REP|https://github.com/apache/incubator-beam/blob/58a029a06aea1030279e5da8f9fa3114f456c1db/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml#L351]: May expose internal representation by returning reference to mutable object Applies to: [AvroUtils$AvroMetadata.getSyncMarker|https://github.com/apache/incubator-beam/blob/58a029a06aea1030279e5da8f9fa3114f456c1db/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AvroUtils.java#L92]. This is a good starter bug. When fixing, please remove the corresponding entries from [findbugs-filter.xml|https://github.com/apache/incubator-beam/blob/master/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml] and verify the build passes. > BitSetCoder transient field byteArrayCoder not set after deserialization > > > Key: BEAM-418 > URL: https://issues.apache.org/jira/browse/BEAM-418 > Project: Beam > Issue Type: Bug > Components: sdk-java-core >Reporter: Scott Wegner >Priority: Minor > Labels: findbugs, newbie, starter > > [FindBugs > SE_TRANSIENT_FIELD_NOT_RESTORED|https://github.com/apache/incubator-beam/blob/58a029a06aea1030279e5da8f9fa3114f456c1db/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml#L357]: > Transient field that isn't set by deserialization. > Applies to: > [BitSetCoder.byteArrayCoder|https://github.com/apache/incubator-beam/blob/58a029a06aea1030279e5da8f9fa3114f456c1db/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BitSetCoder.java#L35]. > This is a good starter bug. When fixing, please remove the corresponding > entries from > [findbugs-filter.xml|https://github.com/apache/incubator-beam/blob/master/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml] > and verify the build passes. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (BEAM-417) AvroUtils$AvroMetadata exposes mutable syncMarker
[ https://issues.apache.org/jira/browse/BEAM-417?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Scott Wegner updated BEAM-417: -- Description: [FindBugs EI_EXPOSE_REP|https://github.com/apache/incubator-beam/blob/58a029a06aea1030279e5da8f9fa3114f456c1db/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml#L351]: May expose internal representation by returning reference to mutable object Applies to: [AvroUtils$AvroMetadata.getSyncMarker|https://github.com/apache/incubator-beam/blob/58a029a06aea1030279e5da8f9fa3114f456c1db/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AvroUtils.java#L92]. This is a good starter bug. When fixing, please remove the corresponding entries from [findbugs-filter.xml|https://github.com/apache/incubator-beam/blob/master/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml] and verify the build passes. was: [FindBugs FE_FLOATING_POINT_EQUALITY|https://github.com/apache/incubator-beam/blob/58a029a06aea1030279e5da8f9fa3114f456c1db/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml#L298]: Test for floating point equality Applies to: [Mean$CountSum.equals|https://github.com/apache/incubator-beam/blob/58a029a06aea1030279e5da8f9fa3114f456c1db/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Mean.java#L165]. This is a good starter bug. When fixing, please remove the corresponding entries from [findbugs-filter.xml|https://github.com/apache/incubator-beam/blob/master/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml] and verify the build passes. > AvroUtils$AvroMetadata exposes mutable syncMarker > - > > Key: BEAM-417 > URL: https://issues.apache.org/jira/browse/BEAM-417 > Project: Beam > Issue Type: Bug > Components: sdk-java-core >Reporter: Scott Wegner >Priority: Minor > Labels: findbugs, newbie, starter > > [FindBugs > EI_EXPOSE_REP|https://github.com/apache/incubator-beam/blob/58a029a06aea1030279e5da8f9fa3114f456c1db/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml#L351]: > May expose internal representation by returning reference to mutable object > Applies to: > [AvroUtils$AvroMetadata.getSyncMarker|https://github.com/apache/incubator-beam/blob/58a029a06aea1030279e5da8f9fa3114f456c1db/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AvroUtils.java#L92]. > This is a good starter bug. When fixing, please remove the corresponding > entries from > [findbugs-filter.xml|https://github.com/apache/incubator-beam/blob/master/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml] > and verify the build passes. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (BEAM-145) OutputTimeFn#assignOutputTime overrides WindowFn#getOutputTime in unfortunate ways
[ https://issues.apache.org/jira/browse/BEAM-145?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15359267#comment-15359267 ] ASF GitHub Bot commented on BEAM-145: - GitHub user kennknowles opened a pull request: https://github.com/apache/incubator-beam/pull/573 [BEAM-145] Fix timestamps in GroupAlsoByWindowsProperties Be sure to do all of the following to help us incorporate your contribution quickly and easily: - [x] Make sure the PR title is formatted like: `[BEAM-] Description of pull request` - [x] Make sure tests pass via `mvn clean verify`. (Even better, enable Travis-CI on your fork and ensure the whole test matrix passes). - [x] Replace `` in the title with the actual Jira issue number, if there is one. - [x] If this contribution is large, please file an Apache [Individual Contributor License Agreement](https://www.apache.org/licenses/icla.txt). --- Some of the timestamps were not adjusted when [BEAM-145](https://issues.apache.org/jira/browse/BEAM-145) was fixed to respect the `WindowFn`'s timestamps. You can merge this pull request into a Git repository by running: $ git pull https://github.com/kennknowles/incubator-beam GABWProperties Alternatively you can review and apply these changes as the patch at: https://github.com/apache/incubator-beam/pull/573.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #573 commit 190740f7971be506df5416dc4dfc5714bba0c972 Author: Kenneth KnowlesDate: 2016-07-01T16:53:37Z Fix timestamps in GroupAlsoByWindowsProperties Some of the timestamps were not adjusted when BEAM-145 was fixed to respect the WindowFn's timestamps. > OutputTimeFn#assignOutputTime overrides WindowFn#getOutputTime in unfortunate > ways > -- > > Key: BEAM-145 > URL: https://issues.apache.org/jira/browse/BEAM-145 > Project: Beam > Issue Type: Bug > Components: runner-core >Reporter: Kenneth Knowles >Assignee: Kenneth Knowles >Priority: Minor > Labels: windowing > Fix For: 0.1.0-incubating > > > Today the {{OutputTimeFn}} includes {{#assignOutputTime}}, {{#combine}}, and > {{#merge}}. Together these express the grouping of timestamps, analogous to > the grouping of values in a GBK / Combine, in a canonical way. > The default {{OutputTimeFn}} is provided by the {{WindowFn}}. In particular, > {{SlidingWindows}} provides an {{OutputTimeFn}} that shifts input timestamps > later to avoid watermark stuckness and then takes the minimum to compute the > output timestamp. > The SDK additionally provides instance for "min", "max" and "end of window" > output timestamps. > Unfortunately, if one overrides the {{OutputTimeFn}} to one of these, the > shifting done by {{SlidingWindows}} is lost. > This is actually only a minor problem for now, since "min" is the default, > "end of window" is unaffected, and "max" has only esoteric uses.The fix is > easy: > This is interrelated with another suggested change: Since there are only > three common {{OutputTimeFn}} instances, and it is a high bandwidth API, it > does not seem worthwhile to leave it in userland. So it is proposed to reduce > it to an enum, which would leave only the {{WindowFn}} as a userland place > for timestamp adjustments. (requiring special casing for end-of-window, since > it cannot be implemented without owning {{#assignOutputTime}}) -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[GitHub] incubator-beam pull request #573: [BEAM-145] Fix timestamps in GroupAlsoByWi...
GitHub user kennknowles opened a pull request: https://github.com/apache/incubator-beam/pull/573 [BEAM-145] Fix timestamps in GroupAlsoByWindowsProperties Be sure to do all of the following to help us incorporate your contribution quickly and easily: - [x] Make sure the PR title is formatted like: `[BEAM-] Description of pull request` - [x] Make sure tests pass via `mvn clean verify`. (Even better, enable Travis-CI on your fork and ensure the whole test matrix passes). - [x] Replace `` in the title with the actual Jira issue number, if there is one. - [x] If this contribution is large, please file an Apache [Individual Contributor License Agreement](https://www.apache.org/licenses/icla.txt). --- Some of the timestamps were not adjusted when [BEAM-145](https://issues.apache.org/jira/browse/BEAM-145) was fixed to respect the `WindowFn`'s timestamps. You can merge this pull request into a Git repository by running: $ git pull https://github.com/kennknowles/incubator-beam GABWProperties Alternatively you can review and apply these changes as the patch at: https://github.com/apache/incubator-beam/pull/573.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #573 commit 190740f7971be506df5416dc4dfc5714bba0c972 Author: Kenneth KnowlesDate: 2016-07-01T16:53:37Z Fix timestamps in GroupAlsoByWindowsProperties Some of the timestamps were not adjusted when BEAM-145 was fixed to respect the WindowFn's timestamps. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[jira] [Commented] (BEAM-416) Jenkins Python Verify post commit tests are timing out
[ https://issues.apache.org/jira/browse/BEAM-416?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15359245#comment-15359245 ] Ahmet Altay commented on BEAM-416: -- This is probably related: > Jenkins Python Verify post commit tests are timing out > -- > > Key: BEAM-416 > URL: https://issues.apache.org/jira/browse/BEAM-416 > Project: Beam > Issue Type: Bug > Components: sdk-py >Reporter: Ahmet Altay >Assignee: Silviu Calinoiu > Attachments: job_output (2) > > > beam_PostCommit_PythonVerify is timing out at the e2e testing phase: > Console output: > https://builds.apache.org/view/Beam/job/beam_PostCommit_PythonVerify/8/console > e2e test: > https://pantheon.corp.google.com/dataflow/job/2016-07-01_08_02_45-15435546446836030984?project=apache-beam-testing > Workers are failing to find the correct container image (from worker logs): > Error syncing pod 6d3e3a71409d65aa43494143d705455b, skipping: failed to > "StartContainer" for "python" with ImagePullBackOff: "Back-off pulling image > \"dataflow.gcr.io/v1beta3/python:latest\"" > It might be related to this commit: > https://github.com/apache/incubator-beam/commit/0bda677d47d5bd5d9c45b74e00e5c3fd113a4f81 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (BEAM-416) Jenkins Python Verify post commit tests are timing out
[ https://issues.apache.org/jira/browse/BEAM-416?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ahmet Altay updated BEAM-416: - Attachment: job_output (2) job output from the timing out job > Jenkins Python Verify post commit tests are timing out > -- > > Key: BEAM-416 > URL: https://issues.apache.org/jira/browse/BEAM-416 > Project: Beam > Issue Type: Bug > Components: sdk-py >Reporter: Ahmet Altay >Assignee: Silviu Calinoiu > Attachments: job_output (2) > > > beam_PostCommit_PythonVerify is timing out at the e2e testing phase: > Console output: > https://builds.apache.org/view/Beam/job/beam_PostCommit_PythonVerify/8/console > e2e test: > https://pantheon.corp.google.com/dataflow/job/2016-07-01_08_02_45-15435546446836030984?project=apache-beam-testing > Workers are failing to find the correct container image (from worker logs): > Error syncing pod 6d3e3a71409d65aa43494143d705455b, skipping: failed to > "StartContainer" for "python" with ImagePullBackOff: "Back-off pulling image > \"dataflow.gcr.io/v1beta3/python:latest\"" > It might be related to this commit: > https://github.com/apache/incubator-beam/commit/0bda677d47d5bd5d9c45b74e00e5c3fd113a4f81 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[GitHub] incubator-beam pull request #572: Update surefire plugin to 2.19.1
GitHub user kennknowles opened a pull request: https://github.com/apache/incubator-beam/pull/572 Update surefire plugin to 2.19.1 Be sure to do all of the following to help us incorporate your contribution quickly and easily: - [x] Make sure the PR title is formatted like: `[BEAM-] Description of pull request` - [x] Make sure tests pass via `mvn clean verify`. (Even better, enable Travis-CI on your fork and ensure the whole test matrix passes). - [x] Replace `` in the title with the actual Jira issue number, if there is one. - [x] If this contribution is large, please file an Apache [Individual Contributor License Agreement](https://www.apache.org/licenses/icla.txt). --- I noticed that we were using a very old version of surefire. The latest version is still 6 months old, so nothing like bleeding edge. It adds simple things like excluding tests by fully qualified class name, which is nicer than file path patterns. You can merge this pull request into a Git repository by running: $ git pull https://github.com/kennknowles/incubator-beam surefire Alternatively you can review and apply these changes as the patch at: https://github.com/apache/incubator-beam/pull/572.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #572 commit 37e266e58a6d3da4e0bcf5426983dc130befd387 Author: Kenneth KnowlesDate: 2016-07-01T03:58:09Z Update surefire plugin to 2.19.1 This version is the most recent, still very old (6 mo), and is the first version that allows test exclusion by fully qualified class name. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[jira] [Closed] (BEAM-415) Use AutoAnnotation for SDK Annotations
[ https://issues.apache.org/jira/browse/BEAM-415?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Thomas Groh closed BEAM-415. Resolution: Invalid Fix Version/s: Not applicable We don't actually return any annotation type, so this isn't actually required. My mistake. > Use AutoAnnotation for SDK Annotations > -- > > Key: BEAM-415 > URL: https://issues.apache.org/jira/browse/BEAM-415 > Project: Beam > Issue Type: Bug > Components: sdk-java-core >Reporter: Thomas Groh >Assignee: Davor Bonaci >Priority: Minor > Fix For: Not applicable > > > Annotation defines implementation requirements for annotations, including > equals and hashCode implementations. Using AutoAnnotation corrects our > implementations. > https://docs.oracle.com/javase/7/docs/api/java/lang/annotation/Annotation.html -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Resolved] (BEAM-392) Update flink runner to use flink version 1.0.3
[ https://issues.apache.org/jira/browse/BEAM-392?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aljoscha Krettek resolved BEAM-392. --- Resolution: Fixed Fix Version/s: 0.2.0-incubating > Update flink runner to use flink version 1.0.3 > -- > > Key: BEAM-392 > URL: https://issues.apache.org/jira/browse/BEAM-392 > Project: Beam > Issue Type: Improvement > Components: runner-flink >Reporter: Ismaël Mejía >Priority: Trivial > Fix For: 0.2.0-incubating > > > Flink has released a new minor stable version not included in Beam yet. Some > of the issues solved in Flink could benefit the users of the Beam runner. -- This message was sent by Atlassian JIRA (v6.3.4#6332)