Repository: beam Updated Branches: refs/heads/master eecbcc48b -> 77c7505b8
DoFnSignature: deprecate isStateful in favor of usesState Project: http://git-wip-us.apache.org/repos/asf/beam/repo Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/ed434ecf Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/ed434ecf Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/ed434ecf Branch: refs/heads/master Commit: ed434ecfdbcfc339815c85aa15852124bb2d3981 Parents: eecbcc4 Author: Kenneth Knowles <k...@google.com> Authored: Fri Jan 20 11:28:45 2017 -0800 Committer: Kenneth Knowles <k...@google.com> Committed: Fri Jan 20 11:35:10 2017 -0800 ---------------------------------------------------------------------- .../beam/runners/dataflow/DataflowPipelineTranslator.java | 2 +- .../apache/beam/sdk/transforms/reflect/DoFnSignature.java | 10 +++++++--- 2 files changed, 8 insertions(+), 4 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/beam/blob/ed434ecf/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java ---------------------------------------------------------------------- diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java index 7609745..cfb3d1a 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java @@ -943,7 +943,7 @@ public class DataflowPipelineTranslator { DoFnInfo.forFn( fn, windowingStrategy, sideInputs, inputCoder, mainOutput, outputMap)))); - if (signature.isStateful()) { + if (signature.usesState()) { stepContext.addInput(PropertyNames.USES_KEYED_STATE, "true"); } } http://git-wip-us.apache.org/repos/asf/beam/blob/ed434ecf/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java index ccc9ac3..5255adc 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java @@ -105,13 +105,17 @@ public abstract class DoFnSignature { @Nullable public abstract Map<String, OnTimerMethod> onTimerMethods(); - /** - * Whether the {@link DoFn} described by this signature uses state. - */ + /** @deprecated use {@link #usesState()}, it's cleaner */ + @Deprecated public boolean isStateful() { return stateDeclarations().size() > 0; } + /** Whether the {@link DoFn} described by this signature uses state. */ + public boolean usesState() { + return stateDeclarations().size() > 0; + } + static Builder builder() { return new AutoValue_DoFnSignature.Builder(); }