Use the correct State class in DoFnSignatures
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/00c7587e Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/00c7587e Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/00c7587e Branch: refs/heads/master Commit: 00c7587e856bdccc35440e08c2a9ed1245ce2d3d Parents: ff6301b Author: Kenneth Knowles <[email protected]> Authored: Fri Oct 14 10:51:57 2016 -0700 Committer: Kenneth Knowles <[email protected]> Committed: Thu Oct 20 11:47:40 2016 -0700 ---------------------------------------------------------------------- .../org/apache/beam/sdk/transforms/reflect/DoFnSignature.java | 6 +++--- .../org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/00c7587e/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 e54b361..0d503d2 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 @@ -25,12 +25,12 @@ import java.util.Collections; import java.util.List; import java.util.Map; import javax.annotation.Nullable; -import javax.swing.plaf.nimbus.State; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFn.ProcessContinuation; import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; import org.apache.beam.sdk.util.TimerSpec; +import org.apache.beam.sdk.util.state.State; import org.apache.beam.sdk.util.state.StateSpec; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptor; @@ -225,10 +225,10 @@ public abstract class DoFnSignature { public abstract static class StateDeclaration { public abstract String id(); public abstract Field field(); - public abstract TypeDescriptor<? extends State<?>> stateType(); + public abstract TypeDescriptor<? extends State> stateType(); static StateDeclaration create( - String id, Field field, TypeDescriptor<? extends State<?>> stateType) { + String id, Field field, TypeDescriptor<? extends State> stateType) { return new AutoValue_DoFnSignature_StateDeclaration(id, field, stateType); } } http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/00c7587e/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java index d11050c..04f50d3 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java @@ -41,7 +41,6 @@ import java.util.LinkedHashSet; import java.util.List; import java.util.Map; import javax.annotation.Nullable; -import javax.swing.plaf.nimbus.State; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.reflect.DoFnSignature.OnTimerMethod; @@ -51,6 +50,7 @@ import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.TimerSpec; import org.apache.beam.sdk.util.common.ReflectHelpers; +import org.apache.beam.sdk.util.state.State; import org.apache.beam.sdk.util.state.StateSpec; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptor; @@ -805,8 +805,8 @@ public class DoFnSignatures { Type stateSpecType = field.getGenericType(); // By static typing this is already a well-formed State subclass - TypeDescriptor<? extends State<?>> stateType = - (TypeDescriptor<? extends State<?>>) + TypeDescriptor<? extends State> stateType = + (TypeDescriptor<? extends State>) TypeDescriptor.of(fnClazz) .resolveType( TypeDescriptor.of(
