Repository: beam
Updated Branches:
  refs/heads/master e686286f1 -> 5fb30ec82


NonNull by default in sdk/transforms/windowing


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/d8353b70
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/d8353b70
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/d8353b70

Branch: refs/heads/master
Commit: d8353b70143be4eb31298c120c9faf4a372beb37
Parents: 90be820
Author: Kenneth Knowles <[email protected]>
Authored: Mon Oct 23 19:17:00 2017 -0700
Committer: Kenneth Knowles <[email protected]>
Committed: Sat Oct 28 08:42:29 2017 -0700

----------------------------------------------------------------------
 .../windowing/IncompatibleWindowException.java  |  2 +-
 .../MergeOverlappingIntervalWindows.java        |  4 +++-
 .../beam/sdk/transforms/windowing/Trigger.java  | 20 +++++++++++---------
 .../beam/sdk/transforms/windowing/Window.java   |  1 +
 .../sdk/transforms/windowing/package-info.java  |  4 ++++
 5 files changed, 20 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/d8353b70/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/IncompatibleWindowException.java
----------------------------------------------------------------------
diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/IncompatibleWindowException.java
 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/IncompatibleWindowException.java
index b7b96ad..20746af 100644
--- 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/IncompatibleWindowException.java
+++ 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/IncompatibleWindowException.java
@@ -32,7 +32,7 @@ public class IncompatibleWindowException extends Exception {
 
   @Override
   public String getMessage() {
-    String windowFn = givenWindowFn == null ? "null" : 
givenWindowFn.getClass().getSimpleName();
+    String windowFn = givenWindowFn.getClass().getSimpleName();
     return String.format("The given WindowFn is %s. %s", windowFn, reason);
   }
 }

http://git-wip-us.apache.org/repos/asf/beam/blob/d8353b70/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/MergeOverlappingIntervalWindows.java
----------------------------------------------------------------------
diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/MergeOverlappingIntervalWindows.java
 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/MergeOverlappingIntervalWindows.java
index 0a68021..0421868 100644
--- 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/MergeOverlappingIntervalWindows.java
+++ 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/MergeOverlappingIntervalWindows.java
@@ -21,6 +21,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
+import javax.annotation.Nullable;
 import org.apache.beam.sdk.annotations.Internal;
 
 /**
@@ -61,9 +62,10 @@ public class MergeOverlappingIntervalWindows {
   }
 
   private static class MergeCandidate {
-    private IntervalWindow union;
+    @Nullable private IntervalWindow union;
     private final List<IntervalWindow> parts;
     public MergeCandidate() {
+      union = null;
       parts = new ArrayList<>();
     }
     public MergeCandidate(IntervalWindow window) {

http://git-wip-us.apache.org/repos/asf/beam/blob/d8353b70/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Trigger.java
----------------------------------------------------------------------
diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Trigger.java
 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Trigger.java
index 519ab67..6985565 100644
--- 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Trigger.java
+++ 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Trigger.java
@@ -23,6 +23,7 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import java.util.Objects;
+import javax.annotation.Nullable;
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.annotations.Internal;
 import org.apache.beam.sdk.transforms.GroupByKey;
@@ -71,9 +72,9 @@ import org.joda.time.Instant;
 @Experimental(Experimental.Kind.TRIGGER)
 public abstract class Trigger implements Serializable {
 
-  protected final List<Trigger> subTriggers;
+  @Nullable protected final List<Trigger> subTriggers;
 
-  protected Trigger(List<Trigger> subTriggers) {
+  protected Trigger(@Nullable List<Trigger> subTriggers) {
     this.subTriggers = subTriggers;
   }
 
@@ -107,15 +108,16 @@ public abstract class Trigger implements Serializable {
   }
 
   /**
-   * Subclasses should override this to return the {@link 
#getContinuationTrigger} of this
-   * {@link Trigger}. For convenience, this is provided the continuation 
trigger of each of the
+   * Subclasses should override this to return the {@link 
#getContinuationTrigger} of this {@link
+   * Trigger}. For convenience, this is provided the continuation trigger of 
each of the
    * sub-triggers in the same order as {@link #subTriggers}.
    *
-   * @param continuationTriggers null if {@link #subTriggers} is null, 
otherwise contains the
-   *                             result of {@link #getContinuationTrigger()} 
on each of the
-   *                             subTriggers in the same order.
+   * @param continuationTriggers {@code null} if {@link #subTriggers} is 
{@code null}, otherwise
+   *     contains the result of {@link #getContinuationTrigger()} on each of 
the subTriggers in the
+   *     same order.
    */
-  protected abstract Trigger getContinuationTrigger(List<Trigger> 
continuationTriggers);
+  @Nullable
+  protected abstract Trigger getContinuationTrigger(@Nullable List<Trigger> 
continuationTriggers);
 
   /**
    * <b><i>For internal use only; no backwards-compatibility 
guarantees.</i></b>
@@ -224,7 +226,7 @@ public abstract class Trigger implements Serializable {
    */
   @Internal
   public abstract static class OnceTrigger extends Trigger {
-    protected OnceTrigger(List<Trigger> subTriggers) {
+    protected OnceTrigger(@Nullable List<Trigger> subTriggers) {
       super(subTriggers);
     }
 

http://git-wip-us.apache.org/repos/asf/beam/blob/d8353b70/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java
----------------------------------------------------------------------
diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java
 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java
index 2337798..3ec8136 100644
--- 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java
+++ 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java
@@ -486,6 +486,7 @@ public abstract class Window<T> extends 
PTransform<PCollection<T>, PCollection<T
       original.populateDisplayData(builder);
     }
 
+    @Nullable
     public WindowFn<T, ?> getWindowFn() {
       return updatedStrategy.getWindowFn();
     }

http://git-wip-us.apache.org/repos/asf/beam/blob/d8353b70/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/package-info.java
----------------------------------------------------------------------
diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/package-info.java
 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/package-info.java
index 406e279..332a7b0 100644
--- 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/package-info.java
+++ 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/package-info.java
@@ -47,4 +47,8 @@
  * {@link org.apache.beam.sdk.transforms.windowing.AfterWatermark} for details 
on the
  * watermark.
  */
+@DefaultAnnotation(NonNull.class)
 package org.apache.beam.sdk.transforms.windowing;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;

Reply via email to