[1/2] incubator-beam git commit: Fix logical error in Sample transform javadoc
Repository: incubator-beam Updated Branches: refs/heads/master 7df32928f -> 9cc2c2b7f Fix logical error in Sample transform javadoc Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/4d86379a Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/4d86379a Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/4d86379a Branch: refs/heads/master Commit: 4d86379a0ff02de10eda8272710c08d73fcc647d Parents: 4605051 Author: Scott WegnerAuthored: Mon Mar 28 09:49:54 2016 -0700 Committer: Scott Wegner Committed: Mon Mar 28 09:49:54 2016 -0700 -- .../main/java/com/google/cloud/dataflow/sdk/transforms/Sample.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/4d86379a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/transforms/Sample.java -- diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/transforms/Sample.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/transforms/Sample.java index c5b6e7e..13de40b 100644 --- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/transforms/Sample.java +++ b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/transforms/Sample.java @@ -44,7 +44,7 @@ public class Sample { * produces a new {@code PCollection} containing up to limit * elements of the input {@code PCollection}. * - * If limit is less than or equal to the size of the input + * If limit is greater than or equal to the size of the input * {@code PCollection}, then all the input's elements will be selected. * * All of the elements of the output {@code PCollection} should fit into
[GitHub] incubator-beam pull request: Fix logical error in Sample transform...
Github user asfgit closed the pull request at: https://github.com/apache/incubator-beam/pull/80 --- 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: Closes #71
Closes #71 Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/7df32928 Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/7df32928 Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/7df32928 Branch: refs/heads/master Commit: 7df32928f4db5199e54d7788b3008a0d43f5bd0a Parents: cd0b6ec 13755f9 Author: Dan HalperinAuthored: Mon Mar 28 17:17:16 2016 -0700 Committer: Dan Halperin Committed: Mon Mar 28 17:17:16 2016 -0700 -- .../main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java| 5 + 1 file changed, 5 insertions(+) --
[1/2] incubator-beam git commit: Validate that Pubsub resource names are at least 3 chars
Repository: incubator-beam Updated Branches: refs/heads/master cd0b6ec7d -> 7df32928f Validate that Pubsub resource names are at least 3 chars Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/13755f9a Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/13755f9a Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/13755f9a Branch: refs/heads/master Commit: 13755f9a23b148661004c29b6626ea0e0a74de72 Parents: cd0b6ec Author: Daniel MillsAuthored: Wed Mar 23 17:37:17 2016 -0700 Committer: Dan Halperin Committed: Mon Mar 28 17:16:42 2016 -0700 -- .../main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java| 5 + 1 file changed, 5 insertions(+) -- http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/13755f9a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java -- diff --git a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java index 653b31f..ae7ac0d 100644 --- a/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java +++ b/sdks/java/core/src/main/java/com/google/cloud/dataflow/sdk/io/PubsubIO.java @@ -105,6 +105,7 @@ public class PubsubIO { private static final Pattern PUBSUB_NAME_REGEXP = Pattern.compile("[a-zA-Z][-._~%+a-zA-Z0-9]+"); + private static final int PUBSUB_NAME_MIN_LENGTH = 3; private static final int PUBSUB_NAME_MAX_LENGTH = 255; private static final String SUBSCRIPTION_RANDOM_TEST_PREFIX = "_random/"; @@ -120,6 +121,10 @@ public class PubsubIO { } private static void validatePubsubName(String name) { +if (name.length() < PUBSUB_NAME_MIN_LENGTH) { + throw new IllegalArgumentException( + "Pubsub object name is shorter than 3 characters: " + name); +} if (name.length() > PUBSUB_NAME_MAX_LENGTH) { throw new IllegalArgumentException( "Pubsub object name is longer than 255 characters: " + name);
[GitHub] incubator-beam pull request: Validate that Pubsub resource names a...
Github user asfgit closed the pull request at: https://github.com/apache/incubator-beam/pull/71 --- 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-53) PubSubIO: reimplement in Java
[ https://issues.apache.org/jira/browse/BEAM-53?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15214975#comment-15214975 ] ASF GitHub Bot commented on BEAM-53: GitHub user mshields822 opened a pull request: https://github.com/apache/incubator-beam/pull/85 [BEAM-53] Java-only pub/sub source and sink (streaming only) First step towards supporting pub/sub i/o in any Java runner. Disclaimers: - No integration tests yet. - No unit tests for the source and sink. Propose we support mocking PubsubGrpcClient. - Depends on grpc-pubsub-v1 which is about to be renamed. - Only supports 'application default' credentials, and ignores any GcsOptions flags. - Not yet wired into the 'deault' PubsubIO implementations. - Watermark tracking is heuristic and may introduce late data. But other than that we're ready to go. You can merge this pull request into a Git repository by running: $ git pull https://github.com/mshields822/incubator-beam pubsub Alternatively you can review and apply these changes as the patch at: https://github.com/apache/incubator-beam/pull/85.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 #85 commit dbc39a605bf09f2d1f1cae05de4caaf176abe6c1 Author: Mark ShieldsDate: 2016-03-28T20:30:37Z Initial import commit 4fa0b3d98c302c8b710106610b5aca0721436f87 Author: Mark Shields Date: 2016-03-28T20:37:23Z Initial import II commit 4ba8f304821a09af12e4fd8703d64c16b1349256 Author: Mark Shields Date: 2016-03-28T21:58:49Z Formatting busywork > PubSubIO: reimplement in Java > - > > Key: BEAM-53 > URL: https://issues.apache.org/jira/browse/BEAM-53 > Project: Beam > Issue Type: New Feature > Components: runner-core >Reporter: Daniel Halperin >Assignee: Mark Shields > > PubSubIO is currently only partially implemented in Java: the > DirectPipelineRunner uses a non-scalable API in a single-threaded manner. > In contrast, the DataflowPipelineRunner uses an entirely different code path > implemented in the Google Cloud Dataflow service. > We need to reimplement PubSubIO in Java in order to support other runners in > a scalable way. > Additionally, we can take this opportunity to add new features: > * getting timestamp from an arbitrary lambda in arbitrary formats rather than > from a message attribute in only 2 formats. > * exposing metadata and attributes in the elements produced by PubSubIO.Read > * setting metadata and attributes in the messages written by PubSubIO.Write -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[GitHub] incubator-beam pull request: [BEAM-53] Java-only pub/sub source an...
GitHub user mshields822 opened a pull request: https://github.com/apache/incubator-beam/pull/85 [BEAM-53] Java-only pub/sub source and sink (streaming only) First step towards supporting pub/sub i/o in any Java runner. Disclaimers: - No integration tests yet. - No unit tests for the source and sink. Propose we support mocking PubsubGrpcClient. - Depends on grpc-pubsub-v1 which is about to be renamed. - Only supports 'application default' credentials, and ignores any GcsOptions flags. - Not yet wired into the 'deault' PubsubIO implementations. - Watermark tracking is heuristic and may introduce late data. But other than that we're ready to go. You can merge this pull request into a Git repository by running: $ git pull https://github.com/mshields822/incubator-beam pubsub Alternatively you can review and apply these changes as the patch at: https://github.com/apache/incubator-beam/pull/85.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 #85 commit dbc39a605bf09f2d1f1cae05de4caaf176abe6c1 Author: Mark ShieldsDate: 2016-03-28T20:30:37Z Initial import commit 4fa0b3d98c302c8b710106610b5aca0721436f87 Author: Mark Shields Date: 2016-03-28T20:37:23Z Initial import II commit 4ba8f304821a09af12e4fd8703d64c16b1349256 Author: Mark Shields Date: 2016-03-28T21:58:49Z Formatting busywork --- 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: Move Model Enforcements into the proper module
Repository: incubator-beam Updated Branches: refs/heads/master f8f3745a8 -> 4f91c2eae Move Model Enforcements into the proper module Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/111e3932 Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/111e3932 Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/111e3932 Branch: refs/heads/master Commit: 111e393219adfd7a41f8545f66a80c638dc38165 Parents: f8f3745 Author: Thomas GrohAuthored: Mon Mar 28 12:02:58 2016 -0700 Committer: Thomas Groh Committed: Mon Mar 28 13:46:04 2016 -0700 -- .../inprocess/AbstractModelEnforcement.java | 36 .../EncodabilityEnforcementFactory.java | 69 -- .../ImmutabilityEnforcementFactory.java | 94 .../sdk/runners/inprocess/ModelEnforcement.java | 61 - .../inprocess/ModelEnforcementFactory.java | 28 -- .../inprocess/AbstractModelEnforcement.java | 36 .../EncodabilityEnforcementFactory.java | 69 ++ .../ImmutabilityEnforcementFactory.java | 94 .../sdk/runners/inprocess/ModelEnforcement.java | 61 + .../inprocess/ModelEnforcementFactory.java | 28 ++ 10 files changed, 288 insertions(+), 288 deletions(-) -- http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/111e3932/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/AbstractModelEnforcement.java -- diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/AbstractModelEnforcement.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/AbstractModelEnforcement.java deleted file mode 100644 index 32b2a67..000 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/AbstractModelEnforcement.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * Copyright (C) 2016 Google Inc. - * - * Licensed 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 com.google.cloud.dataflow.sdk.runners.inprocess; - -import com.google.cloud.dataflow.sdk.runners.inprocess.InProcessPipelineRunner.CommittedBundle; -import com.google.cloud.dataflow.sdk.util.WindowedValue; - -/** - * An abstract {@link ModelEnforcement} that provides default empty implementations for each method. - */ -abstract class AbstractModelEnforcement implements ModelEnforcement { - @Override - public void beforeElement(WindowedValue element) {} - - @Override - public void afterElement(WindowedValue element) {} - - @Override - public void afterFinish( - CommittedBundle input, - InProcessTransformResult result, - Iterable> outputs) {} -} http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/111e3932/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/EncodabilityEnforcementFactory.java -- diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/EncodabilityEnforcementFactory.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/EncodabilityEnforcementFactory.java deleted file mode 100644 index 0e38b55..000 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/runners/inprocess/EncodabilityEnforcementFactory.java +++ /dev/null @@ -1,69 +0,0 @@ -/* - * Copyright (C) 2016 Google Inc. - * - * Licensed 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 com.google.cloud.dataflow.sdk.runners.inprocess; - -import static com.google.common.base.Preconditions.checkArgument; - -import com.google.cloud.dataflow.sdk.coders.Coder; -import
[jira] [Commented] (BEAM-121) Publish DisplayData from common PTransforms
[ https://issues.apache.org/jira/browse/BEAM-121?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15214866#comment-15214866 ] ASF GitHub Bot commented on BEAM-121: - GitHub user swegner opened a pull request: https://github.com/apache/incubator-beam/pull/83 DisplayData tweaks based on transform usage [BEAM-121] Publish DisplayData from common PTransforms * Add boolean-valued display data. * Implement equality for DislpayData.Item * Add ability to override namespace for included subcomponents. * Additional Matchers for testing display data * Update DisplayData inner class privacy You can merge this pull request into a Git repository by running: $ git pull https://github.com/swegner/incubator-beam displaydata-tweaks Alternatively you can review and apply these changes as the patch at: https://github.com/apache/incubator-beam/pull/83.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 #83 commit a52c1b6cdc033f64f526cff9cf0a2c4d8f23aa14 Author: Scott WegnerDate: 2016-03-28T20:38:35Z DisplayData tweaks based on transform usage. * Add boolean-valued display data. * Implement equality for DislpayData.Item * Add ability to override namespace for included subcomponents. * Additional Matchers for testing display data * Update DisplayData inner class privacy > Publish DisplayData from common PTransforms > --- > > Key: BEAM-121 > URL: https://issues.apache.org/jira/browse/BEAM-121 > Project: Beam > Issue Type: Sub-task > Components: sdk-java-core >Reporter: Scott Wegner >Assignee: Scott Wegner > -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[GitHub] incubator-beam pull request: DisplayData tweaks based on transform...
GitHub user swegner opened a pull request: https://github.com/apache/incubator-beam/pull/83 DisplayData tweaks based on transform usage [BEAM-121] Publish DisplayData from common PTransforms * Add boolean-valued display data. * Implement equality for DislpayData.Item * Add ability to override namespace for included subcomponents. * Additional Matchers for testing display data * Update DisplayData inner class privacy You can merge this pull request into a Git repository by running: $ git pull https://github.com/swegner/incubator-beam displaydata-tweaks Alternatively you can review and apply these changes as the patch at: https://github.com/apache/incubator-beam/pull/83.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 #83 commit a52c1b6cdc033f64f526cff9cf0a2c4d8f23aa14 Author: Scott WegnerDate: 2016-03-28T20:38:35Z DisplayData tweaks based on transform usage. * Add boolean-valued display data. * Implement equality for DislpayData.Item * Add ability to override namespace for included subcomponents. * Additional Matchers for testing display data * Update DisplayData inner class privacy --- 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-119) Integrate DisplayData into DataflowPipelineRunner
[ https://issues.apache.org/jira/browse/BEAM-119?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Scott Wegner closed BEAM-119. - Resolution: Fixed This has been merged. > Integrate DisplayData into DataflowPipelineRunner > - > > Key: BEAM-119 > URL: https://issues.apache.org/jira/browse/BEAM-119 > Project: Beam > Issue Type: Sub-task > Components: sdk-java-core >Reporter: Scott Wegner >Assignee: Scott Wegner > -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Closed] (BEAM-118) Add DisplayData to SDK
[ https://issues.apache.org/jira/browse/BEAM-118?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Scott Wegner closed BEAM-118. - Resolution: Fixed This has been merged. > Add DisplayData to SDK > -- > > Key: BEAM-118 > URL: https://issues.apache.org/jira/browse/BEAM-118 > Project: Beam > Issue Type: Sub-task > Components: sdk-java-core >Reporter: Scott Wegner >Assignee: Scott Wegner > -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Closed] (BEAM-90) TestCounterSource needs to include ability to force-fail snapshot
[ https://issues.apache.org/jira/browse/BEAM-90?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mark Shields closed BEAM-90. Resolution: Fixed > TestCounterSource needs to include ability to force-fail snapshot > - > > Key: BEAM-90 > URL: https://issues.apache.org/jira/browse/BEAM-90 > Project: Beam > Issue Type: Bug > Components: testing >Reporter: Mark Shields >Assignee: Mark Shields > > TestCounterSource can be used to test a runner's handling of > UnboundedSources. It should allow an exception to be thrown during > checkpointing so as to also test the runner's resilience to failure. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Closed] (BEAM-97) Input watermarks can never be null
[ https://issues.apache.org/jira/browse/BEAM-97?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mark Shields closed BEAM-97. Resolution: Fixed > Input watermarks can never be null > -- > > Key: BEAM-97 > URL: https://issues.apache.org/jira/browse/BEAM-97 > Project: Beam > Issue Type: Bug > Components: sdk-java-core >Reporter: Mark Shields >Assignee: Mark Shields > > A null input watermark (eg the result of > TimerInternals.currentInputWatermarkTime()) is currently interpreted to mean > 'computation is still starting and watermark is not yet known'. However, the > Google runner was also returning null for 'computation has just restarted and > previous watermark has not been established'. As a result code such as > WatermarkHold would create data holds using the first interpretation which is > unsound for the second interpretation. > Instead, input watermarks should never be null so the interpretation is never > ambiguous. The Google runner is ready to support this. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (BEAM-22) DirectPipelineRunner: support for unbounded collections
[ https://issues.apache.org/jira/browse/BEAM-22?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15214814#comment-15214814 ] ASF GitHub Bot commented on BEAM-22: GitHub user tgroh opened a pull request: https://github.com/apache/incubator-beam/pull/82 [BEAM-22] Execute ModelEnforcements in TransformExecutor This allows a configurable application of Model Enforcement based on the class of transform being executed, both before and after an element is processed and after the transform completes. You can merge this pull request into a Git repository by running: $ git pull https://github.com/tgroh/incubator-beam ippr_model_enforcing_transform_executor Alternatively you can review and apply these changes as the patch at: https://github.com/apache/incubator-beam/pull/82.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 #82 commit b76b9d62f2a21e534238437f0e6aa34321146007 Author: Thomas GrohDate: 2016-03-28T16:35:33Z Execute ModelEnforcements in TransformExecutor This allows a configurable application of Model Enforcement based on the class of transform being executed, both before and after an element is processed and after the transform completes. > DirectPipelineRunner: support for unbounded collections > --- > > Key: BEAM-22 > URL: https://issues.apache.org/jira/browse/BEAM-22 > Project: Beam > Issue Type: Improvement > Components: runner-direct >Reporter: Davor Bonaci >Assignee: Thomas Groh > > DirectPipelineRunner currently runs over bounded PCollections only, and > implements only a portion of the Beam Model. > We should improve it to faithfully implement the full Beam Model, such as add > ability to run over unbounded PCollections, and better resemble execution > model in a distributed system. > This further enables features such as a testing source which may simulate > late data and test triggers in the pipeline. Finally, we may want to expose > an option to select between "debug" (single threaded), "chaos monkey" (test > as many model requirements as possible), and "performance" (multi-threaded). > more testing (chaos monkey) > Once this is done, we should update this StackOverflow question: > http://stackoverflow.com/questions/35350113/testing-triggers-with-processing-time/35401426#35401426 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[GitHub] incubator-beam pull request: [BEAM-22] Execute ModelEnforcements i...
GitHub user tgroh opened a pull request: https://github.com/apache/incubator-beam/pull/82 [BEAM-22] Execute ModelEnforcements in TransformExecutor This allows a configurable application of Model Enforcement based on the class of transform being executed, both before and after an element is processed and after the transform completes. You can merge this pull request into a Git repository by running: $ git pull https://github.com/tgroh/incubator-beam ippr_model_enforcing_transform_executor Alternatively you can review and apply these changes as the patch at: https://github.com/apache/incubator-beam/pull/82.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 #82 commit b76b9d62f2a21e534238437f0e6aa34321146007 Author: Thomas GrohDate: 2016-03-28T16:35:33Z Execute ModelEnforcements in TransformExecutor This allows a configurable application of Model Enforcement based on the class of transform being executed, both before and after an element is processed and after the transform completes. --- 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: Exclude Eclipse .checkstyle files fro...
GitHub user kennknowles opened a pull request: https://github.com/apache/incubator-beam/pull/81 Exclude Eclipse .checkstyle files from RAT check You can merge this pull request into a Git repository by running: $ git pull https://github.com/kennknowles/incubator-beam eclipse-rat Alternatively you can review and apply these changes as the patch at: https://github.com/apache/incubator-beam/pull/81.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 #81 commit 2ea54626eef00bc6c27df53d0171375103e33e58 Author: Kenneth KnowlesDate: 2016-03-28T19:52:02Z Exclude Eclipse .checkstyle files from RAT check --- 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: Add ModelEnforcements
Github user asfgit closed the pull request at: https://github.com/apache/incubator-beam/pull/74 --- 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: Fix logical error in Sample transform...
GitHub user swegner opened a pull request: https://github.com/apache/incubator-beam/pull/80 Fix logical error in Sample transform javadoc If the `limit` passed into the `Sample` transform is *greater than or equal to* the size of the input, then all the input's elements will be selected. You can merge this pull request into a Git repository by running: $ git pull https://github.com/swegner/incubator-beam sample-docs Alternatively you can review and apply these changes as the patch at: https://github.com/apache/incubator-beam/pull/80.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 #80 commit 4d86379a0ff02de10eda8272710c08d73fcc647d Author: Scott WegnerDate: 2016-03-28T16:49:54Z Fix logical error in Sample transform javadoc --- 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: Add GitHub pull request template
Repository: incubator-beam Updated Branches: refs/heads/master 706fc5376 -> 460505175 Add GitHub pull request template Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/ed690c06 Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/ed690c06 Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/ed690c06 Branch: refs/heads/master Commit: ed690c069360791e881a3e487cc3146a8aa280ce Parents: 706fc53 Author: Kenneth KnowlesAuthored: Mon Mar 28 09:30:38 2016 -0700 Committer: Kenneth Knowles Committed: Mon Mar 28 09:30:38 2016 -0700 -- .github/PULL_REQUEST_TEMPLATE.md | 12 1 file changed, 12 insertions(+) -- http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ed690c06/.github/PULL_REQUEST_TEMPLATE.md -- diff --git a/.github/PULL_REQUEST_TEMPLATE.md b/.github/PULL_REQUEST_TEMPLATE.md new file mode 100644 index 000..05b0430 --- /dev/null +++ b/.github/PULL_REQUEST_TEMPLATE.md @@ -0,0 +1,12 @@ +[BEAM-] Description of pull request + +Be sure to do all of the following to help us incorporate your contribution +quickly and easily: + + - [ ] Make sure tests pass via `mvn clean verify`. (Even better, enable + Travis-CI for 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). +
[GitHub] incubator-beam pull request: Add GitHub pull request template
Github user asfgit closed the pull request at: https://github.com/apache/incubator-beam/pull/79 --- 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: Add GitHub pull request template
GitHub user kennknowles opened a pull request: https://github.com/apache/incubator-beam/pull/79 Add GitHub pull request template The contribution guide is still receiving comment, so this is subject to revision. But I think we can get one in place that is somewhat useful immediately. And I am eager to try out this rather fun feature :-) You can merge this pull request into a Git repository by running: $ git pull https://github.com/kennknowles/incubator-beam pull-request-template Alternatively you can review and apply these changes as the patch at: https://github.com/apache/incubator-beam/pull/79.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 #79 commit ed690c069360791e881a3e487cc3146a8aa280ce Author: Kenneth KnowlesDate: 2016-03-28T16:30:38Z Add GitHub pull request template --- 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. ---