Repository: incubator-beam Updated Branches: refs/heads/master 5f0288b44 -> a2b118afe
Fix and file bugs for FindBugs issues Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/3875071b Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/3875071b Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/3875071b Branch: refs/heads/master Commit: 3875071b5289bb729ad84de97b161d679d3c3292 Parents: 1963bde Author: Scott Wegner <[email protected]> Authored: Thu Jun 30 16:16:14 2016 -0700 Committer: Scott Wegner <[email protected]> Committed: Thu Jul 7 09:47:44 2016 -0700 ---------------------------------------------------------------------- .../apache/beam/sdk/util/PaneInfoTracker.java | 4 + .../apache/beam/sdk/util/SystemReduceFn.java | 4 + .../org/apache/beam/sdk/util/TriggerRunner.java | 10 + .../org/apache/beam/sdk/util/WatermarkHold.java | 4 + .../src/main/resources/beam/findbugs-filter.xml | 412 ++----------------- .../apache/beam/sdk/coders/InstantCoder.java | 33 +- .../main/java/org/apache/beam/sdk/io/Write.java | 3 +- .../sdk/options/PipelineOptionsFactory.java | 10 +- .../beam/sdk/testing/CoderProperties.java | 3 +- .../org/apache/beam/sdk/testing/PAssert.java | 4 + .../beam/sdk/testing/SerializableMatchers.java | 4 + .../apache/beam/sdk/transforms/DoFnTester.java | 2 - .../sdk/transforms/display/DisplayData.java | 2 +- .../beam/sdk/transforms/join/CoGbkResult.java | 2 +- .../windowing/AfterDelayFromFirstElement.java | 6 + .../sdk/transforms/windowing/AfterPane.java | 4 + ...AttemptAndTimeBoundedExponentialBackOff.java | 4 + .../sdk/util/ExposedByteArrayInputStream.java | 3 + .../sdk/util/ExposedByteArrayOutputStream.java | 4 + .../apache/beam/sdk/util/PCollectionViews.java | 4 +- .../org/apache/beam/sdk/util/ReleaseInfo.java | 12 +- .../beam/sdk/util/TriggerContextFactory.java | 5 +- .../beam/sdk/util/common/ReflectHelpers.java | 13 +- .../beam/sdk/util/state/StateMerging.java | 18 +- 24 files changed, 137 insertions(+), 433 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3875071b/runners/core-java/src/main/java/org/apache/beam/sdk/util/PaneInfoTracker.java ---------------------------------------------------------------------- diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/PaneInfoTracker.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/PaneInfoTracker.java index 0a47feb..812e99a 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/PaneInfoTracker.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/PaneInfoTracker.java @@ -33,6 +33,8 @@ import com.google.common.annotations.VisibleForTesting; import org.joda.time.Instant; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; + /** * Determine the timing and other properties of a new pane for a given computation, key and window. * Incorporates any previous pane, whether the pane has been produced because an @@ -70,6 +72,8 @@ public class PaneInfoTracker { return new ReadableState<PaneInfo>() { @Override + @SuppressFBWarnings(value = "RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT", + justification = "prefetch side effect") public ReadableState<PaneInfo> readLater() { previousPaneFuture.readLater(); return this; http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3875071b/runners/core-java/src/main/java/org/apache/beam/sdk/util/SystemReduceFn.java ---------------------------------------------------------------------- diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/SystemReduceFn.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/SystemReduceFn.java index 2eeee54..f7dca94 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/SystemReduceFn.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/SystemReduceFn.java @@ -34,6 +34,8 @@ import org.apache.beam.sdk.util.state.StateMerging; import org.apache.beam.sdk.util.state.StateTag; import org.apache.beam.sdk.util.state.StateTags; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; + /** * {@link ReduceFn} implementing the default reduction behaviors of {@link GroupByKey}. * @@ -114,6 +116,8 @@ public abstract class SystemReduceFn<K, InputT, AccumT, OutputT, W extends Bound } @Override + @SuppressFBWarnings(value = "RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT", + justification = "prefetch side effect") public void prefetchOnTrigger(StateAccessor<K> state) { state.access(bufferTag).readLater(); } http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3875071b/runners/core-java/src/main/java/org/apache/beam/sdk/util/TriggerRunner.java ---------------------------------------------------------------------- diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/TriggerRunner.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/TriggerRunner.java index 878d1d7..e00b843 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/TriggerRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/TriggerRunner.java @@ -37,6 +37,8 @@ import java.util.BitSet; import java.util.Collection; import java.util.Map; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; + /** * Executes a trigger while managing persistence of information about which subtriggers are * finished. Subtriggers include all recursive trigger expressions as well as the entire trigger. @@ -100,6 +102,8 @@ public class TriggerRunner<W extends BoundedWindow> { return readFinishedBits(state.access(FINISHED_BITS_TAG)).isFinished(rootTrigger); } + @SuppressFBWarnings(value = "RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT", + justification = "prefetch side effect") public void prefetchForValue(W window, StateAccessor<?> state) { if (isFinishedSetNeeded()) { state.access(FINISHED_BITS_TAG).readLater(); @@ -108,6 +112,8 @@ public class TriggerRunner<W extends BoundedWindow> { contextFactory.createStateAccessor(window, rootTrigger)); } + @SuppressFBWarnings(value = "RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT", + justification = "prefetch side effect") public void prefetchOnFire(W window, StateAccessor<?> state) { if (isFinishedSetNeeded()) { state.access(FINISHED_BITS_TAG).readLater(); @@ -115,6 +121,8 @@ public class TriggerRunner<W extends BoundedWindow> { rootTrigger.getSpec().prefetchOnFire(contextFactory.createStateAccessor(window, rootTrigger)); } + @SuppressFBWarnings(value = "RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT", + justification = "prefetch side effect") public void prefetchShouldFire(W window, StateAccessor<?> state) { if (isFinishedSetNeeded()) { state.access(FINISHED_BITS_TAG).readLater(); @@ -137,6 +145,8 @@ public class TriggerRunner<W extends BoundedWindow> { persistFinishedSet(state, finishedSet); } + @SuppressFBWarnings(value = "RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT", + justification = "prefetch side effect") public void prefetchForMerge( W window, Collection<W> mergingWindows, MergingStateAccessor<?, W> state) { if (isFinishedSetNeeded()) { http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3875071b/runners/core-java/src/main/java/org/apache/beam/sdk/util/WatermarkHold.java ---------------------------------------------------------------------- diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/WatermarkHold.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/WatermarkHold.java index 61ab44a..985f210 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/WatermarkHold.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/WatermarkHold.java @@ -40,6 +40,8 @@ import java.io.Serializable; import javax.annotation.Nullable; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; + /** * Implements the logic to hold the output watermark for a computation back * until it has seen all the elements it needs based on the input watermark for the @@ -465,6 +467,8 @@ class WatermarkHold<W extends BoundedWindow> implements Serializable { final WatermarkHoldState<BoundedWindow> extraHoldState = context.state().access(EXTRA_HOLD_TAG); return new ReadableState<OldAndNewHolds>() { @Override + @SuppressFBWarnings(value = "RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT", + justification = "") public ReadableState<OldAndNewHolds> readLater() { elementHoldState.readLater(); extraHoldState.readLater(); http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3875071b/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml ---------------------------------------------------------------------- diff --git a/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml b/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml index d151315..1226cd1 100644 --- a/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml +++ b/sdks/java/build-tools/src/main/resources/beam/findbugs-filter.xml @@ -19,6 +19,8 @@ <!-- Ignored bug categories. Bugs in these categories should not block the build. --> <Bug category="I18N"/> <Bug pattern="DM_STRING_CTOR"/> + <Bug pattern="EI_EXPOSE_REP" /> + <Bug pattern="EI_EXPOSE_REP2" /> <Bug pattern="SE_NO_SERIALVERSIONID"/> @@ -31,513 +33,155 @@ <Class name="org.apache.beam.sdk.coders.AvroCoder"/> <Field name="decoder"/> <Bug pattern="SE_BAD_FIELD"/> - <!--Non-transient non-serializable instance field in serializable class--> + <!--[BEAM-394] Non-transient non-serializable instance field in serializable class--> </Match> <Match> <Class name="org.apache.beam.sdk.coders.AvroCoder"/> <Field name="encoder"/> <Bug pattern="SE_BAD_FIELD"/> - <!--Non-transient non-serializable instance field in serializable class--> + <!--[BEAM-394] Non-transient non-serializable instance field in serializable class--> </Match> <Match> <Class name="org.apache.beam.sdk.coders.AvroCoder"/> <Field name="reader"/> <Bug pattern="SE_BAD_FIELD"/> - <!--Non-transient non-serializable instance field in serializable class--> + <!--[BEAM-394] Non-transient non-serializable instance field in serializable class--> </Match> <Match> <Class name="org.apache.beam.sdk.coders.AvroCoder"/> <Field name="writer"/> <Bug pattern="SE_BAD_FIELD"/> - <!--Non-transient non-serializable instance field in serializable class--> + <!--[BEAM-394] Non-transient non-serializable instance field in serializable class--> </Match> <Match> <Class name="org.apache.beam.sdk.coders.AvroCoder"/> <Field name="reader"/> <Bug pattern="SE_BAD_FIELD_STORE"/> - <!--Non-serializable value stored into instance field of a serializable class--> + <!--[BEAM-394] Non-serializable value stored into instance field of a serializable class--> </Match> <Match> <Class name="org.apache.beam.sdk.coders.AvroCoder"/> <Field name="writer"/> <Bug pattern="SE_BAD_FIELD_STORE"/> - <!--Non-serializable value stored into instance field of a serializable class--> + <!--[BEAM-394] Non-serializable value stored into instance field of a serializable class--> </Match> <Match> <Class name="org.apache.beam.sdk.coders.Coder$NonDeterministicException"/> <Bug pattern="NM_CLASS_NOT_EXCEPTION"/> - <!--Class is not derived from an Exception, even though it is named as such--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.coders.EntityCoder"/> - <Method name="decode"/> - <Bug pattern="RR_NOT_CHECKED"/> - <!--Method ignores results of InputStream.read()--> + <!--[BEAM-396] Class is not derived from an Exception, even though it is named as such--> </Match> <Match> <Class name="org.apache.beam.sdk.coders.JAXBCoder"/> <Method name="getContext"/> <Bug pattern="DC_DOUBLECHECK"/> - <!--Possible double check of field--> + <!--[BEAM-398] Possible double check of field--> </Match> <Match> <Class name="org.apache.beam.sdk.coders.JAXBCoder"/> <Field name="jaxbContext"/> <Bug pattern="IS2_INCONSISTENT_SYNC"/> - <!--Inconsistent synchronization--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.coders.InstantCoder$1"/> - <Bug pattern="HE_INHERITS_EQUALS_USE_HASHCODE"/> - <!--Class inherits equals() and uses Object.hashCode()--> + <!--[BEAM-398] Inconsistent synchronization--> </Match> <Match> <Class name="org.apache.beam.sdk.coders.SerializableCoder"/> <Method name="equals"/> <Bug pattern="NP_EQUALS_SHOULD_HANDLE_NULL_ARGUMENT"/> - <!--equals() method does not check for null argument--> + <!--[BEAM-399] equals() method does not check for null argument--> </Match> <Match> <Class name="org.apache.beam.sdk.coders.StandardCoder"/> <Method name="toString"/> <Bug pattern="SBSC_USE_STRINGBUFFER_CONCATENATION"/> - <!--Method invokes inefficient new String(String) constructor--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.coders.StructuralByteArray"/> - <Method name="getValue"/> - <Bug pattern="EI_EXPOSE_REP"/> - <!--May expose internal representation by returning reference to mutable object--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.coders.StructuralByteArray"/> - <Method name="<init>"/> - <Bug pattern="EI_EXPOSE_REP2"/> - <!--May expose internal representation by incorporating reference to mutable object--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.io.PubsubIO$Read$Bound$PubsubBoundedReader"/> - <Method name="processElement"/> - <Bug pattern="DMI_RANDOM_USED_ONLY_ONCE"/> - <!--Random object created and used only once--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.io.PubsubUnboundedSink$WriterFn"/> - <Method name="publishBatch"/> - <Bug pattern="DLS_DEAD_LOCAL_STORE"/> - <!--Dead store to local variable--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.io.Write$Bound"/> - <Method name="createWrite"/> - <Bug pattern="DLS_DEAD_LOCAL_STORE"/> - <!--Dead store to local variable--> + <!--[BEAM-401] Method invokes inefficient new String(String) constructor--> </Match> <Match> <Class name="org.apache.beam.sdk.io.range.OffsetRangeTracker"/> <Field name="done"/> <Bug pattern="IS2_INCONSISTENT_SYNC"/> - <!--Inconsistent synchronization--> + <!--[BEAM-407] Inconsistent synchronization--> </Match> <Match> <Class name="org.apache.beam.sdk.io.range.OffsetRangeTracker"/> <Field name="lastRecordStart"/> <Bug pattern="IS2_INCONSISTENT_SYNC"/> - <!--Inconsistent synchronization--> + <!--[BEAM-407] Inconsistent synchronization--> </Match> <Match> <Class name="org.apache.beam.sdk.io.range.OffsetRangeTracker"/> <Field name="offsetOfLastSplitPoint"/> <Bug pattern="IS2_INCONSISTENT_SYNC"/> - <!--Inconsistent synchronization--> + <!--[BEAM-407] Inconsistent synchronization--> </Match> <Match> <Class name="org.apache.beam.sdk.io.range.OffsetRangeTracker"/> <Field name="splitPointsSeen"/> <Bug pattern="IS2_INCONSISTENT_SYNC"/> - <!--Inconsistent synchronization--> + <!--[BEAM-407] Inconsistent synchronization--> </Match> <Match> <Class name="org.apache.beam.sdk.io.range.OffsetRangeTracker"/> <Field name="startOffset"/> <Bug pattern="IS2_INCONSISTENT_SYNC"/> - <!--Inconsistent synchronization--> + <!--[BEAM-407] Inconsistent synchronization--> </Match> <Match> <Class name="org.apache.beam.sdk.io.range.OffsetRangeTracker"/> <Field name="stopOffset"/> <Bug pattern="IS2_INCONSISTENT_SYNC"/> - <!--Inconsistent synchronization--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.options.PipelineOptionsFactory$1"/> - <Method name="apply"/> - <Bug pattern="NP_PARAMETER_MUST_BE_NONNULL_BUT_MARKED_AS_NULLABLE"/> - <!--Parameter must be non-null but is marked as nullable--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.options.PipelineOptionsFactory$2"/> - <Method name="apply"/> - <Bug pattern="NP_PARAMETER_MUST_BE_NONNULL_BUT_MARKED_AS_NULLABLE"/> - <!--Parameter must be non-null but is marked as nullable--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.options.PipelineOptionsFactory$3"/> - <Method name="apply"/> - <Bug pattern="NP_PARAMETER_MUST_BE_NONNULL_BUT_MARKED_AS_NULLABLE"/> - <!--Parameter must be non-null but is marked as nullable--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.options.PipelineOptionsFactory$4"/> - <Method name="apply"/> - <Bug pattern="NP_PARAMETER_MUST_BE_NONNULL_BUT_MARKED_AS_NULLABLE"/> - <!--Parameter must be non-null but is marked as nullable--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.runners.DirectPipelineRunner"/> - <Method name="<init>"/> - <Bug pattern="DMI_RANDOM_USED_ONLY_ONCE"/> - <!--Random object created and used only once--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.testing.CoderProperties"/> - <Field name="ALL_CONTEXTS"/> - <Bug pattern="MS_MUTABLE_COLLECTION_PKGPROTECT"/> - <!--Field is a mutable collection which should be package protected--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.testing.PAssert$IterableAssert"/> - <Method name="equals"/> - <Bug pattern="EQ_UNUSUAL"/> - <!--Unusual equals method --> - </Match> - <Match> - <Class name="org.apache.beam.sdk.testing.PAssert$IterableAssert"/> - <Field name="pipeline"/> - <Bug pattern="SE_BAD_FIELD"/> - <!--Non-transient non-serializable instance field in serializable class--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.testing.PAssert$PCollectionContentsAssert"/> - <Method name="equals"/> - <Bug pattern="EQ_UNUSUAL"/> - <!--Unusual equals method--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.testing.PAssert$PCollectionViewAssert"/> - <Method name="equals"/> - <Bug pattern="EQ_UNUSUAL"/> - <!--Unusual equals method--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.testing.PAssert$SingletonAssert"/> - <Method name="equals"/> - <Bug pattern="EQ_UNUSUAL"/> - <!--Unusual equals method --> - </Match> - <Match> - <Class name="org.apache.beam.sdk.testing.PAssert$SingletonAssert"/> - <Field name="pipeline"/> - <Bug pattern="SE_BAD_FIELD"/> - <!--Non-transient non-serializable instance field in serializable class--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.testing.SerializableMatchers$SerializableArrayViaCoder"/> - <Field name="value"/> - <Bug pattern="SE_TRANSIENT_FIELD_NOT_RESTORED"/> - <!--Transient field that isn't set by deserialization. --> + <!--[BEAM-407] Inconsistent synchronization--> </Match> <Match> <Class name="org.apache.beam.sdk.testing.WindowSupplier"/> <Field name="windows"/> <Bug pattern="IS2_INCONSISTENT_SYNC"/> - <!--Inconsistent synchronization --> + <!--[BEAM-409] Inconsistent synchronization --> </Match> <Match> <Class name="org.apache.beam.sdk.transforms.ApproximateQuantiles$ApproximateQuantilesCombineFn"/> <Method name="create"/> <Bug pattern="ICAST_INT_CAST_TO_DOUBLE_PASSED_TO_CEIL"/> - <!--Integral value cast to double and then passed to Math.ceil--> + <!--[BEAM-409] Integral value cast to double and then passed to Math.ceil--> </Match> <Match> <Class name="org.apache.beam.sdk.transforms.ApproximateQuantiles$QuantileBuffer"/> <Method name="compareTo"/> <Bug pattern="EQ_COMPARETO_USE_OBJECT_EQUALS"/> - <!--Class defines compareTo(...) and uses Object.equals()--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.transforms.Create$TimestampedValues"/> - <Field name="elems"/> - <Bug pattern="SE_TRANSIENT_FIELD_NOT_RESTORED"/> - <!--Transient field that isn't set by deserialization. --> - </Match> - <Match> - <Class name="org.apache.beam.sdk.transforms.DoFnTester"/> - <Field name="sideOutputTags"/> - <Bug pattern="URF_UNREAD_FIELD"/> - <!--Unread field--> + <!--[BEAM-410] Class defines compareTo(...) and uses Object.equals()--> </Match> <Match> <Class name="org.apache.beam.sdk.transforms.Mean$CountSum"/> <Method name="equals"/> <Bug pattern="FE_FLOATING_POINT_EQUALITY"/> - <!--Test for floating point equality--> + <!--[BEAM-413] Test for floating point equality--> </Match> <Match> <Class name="org.apache.beam.sdk.transforms.display.DisplayData"/> <Field name="entries"/> <Bug pattern="SE_BAD_FIELD"/> - <!--Non-transient non-serializable instance field in serializable class--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.transforms.display.DisplayData$InternalBuilder$PopulateDisplayDataException"/> - <Bug pattern="SIC_INNER_SHOULD_BE_STATIC"/> - <!--Should be a static inner class--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.transforms.join.CoGbkResult$UnionValueIterator"/> - <Method name="hasNext"/> - <Bug pattern="RC_REF_COMPARISON_BAD_PRACTICE_BOOLEAN"/> - <!--Suspicious reference comparison of Boolean values--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.transforms.windowing.AfterDelayFromFirstElement"/> - <Method name="prefetchOnElement"/> - <Bug pattern="RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT"/> - <!--Return value of method without side effect is ignored--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.transforms.windowing.AfterDelayFromFirstElement"/> - <Method name="prefetchShouldFire"/> - <Bug pattern="RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT"/> - <!--Return value of method without side effect is ignored--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.transforms.windowing.AfterPane"/> - <Method name="prefetchShouldFire"/> - <Bug pattern="RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT"/> - <!--Return value of method without side effect is ignored--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.util.AttemptAndTimeBoundedExponentialBackOff"/> - <Method name="reset"/> - <Bug pattern="UR_UNINIT_READ_CALLED_FROM_SUPER_CONSTRUCTOR"/> - <!--Uninitialized read of field method called from constructor of superclass--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.util.AttemptAndTimeBoundedExponentialBackOff"/> - <Method name="reset"/> - <Bug pattern="UR_UNINIT_READ_CALLED_FROM_SUPER_CONSTRUCTOR"/> - <!--Uninitialized read of field method called from constructor of superclass--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.util.AvroUtils$AvroMetadata"/> - <Method name="getSyncMarker"/> - <Bug pattern="EI_EXPOSE_REP"/> - <!--May expose internal representation by returning reference to mutable object--> + <!--[BEAM-412] Non-transient non-serializable instance field in serializable class--> </Match> <Match> <Class name="org.apache.beam.sdk.util.BitSetCoder"/> <Field name="byteArrayCoder"/> <Bug pattern="SE_TRANSIENT_FIELD_NOT_RESTORED"/> - <!--Transient field that isn't set by deserialization. --> + <!--[BEAM-418] Transient field that isn't set by deserialization. --> </Match> <Match> <Class name="org.apache.beam.sdk.util.CombineFnUtil$NonSerializableBoundedKeyedCombineFn"/> <Field name="context"/> <Bug pattern="SE_BAD_FIELD"/> - <!--Non-transient non-serializable instance field in serializable class--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.util.ExposedByteArrayInputStream"/> - <Method name="readAll"/> - <Bug pattern="EI_EXPOSE_REP"/> - <!--May expose internal representation by returning reference to mutable object--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.util.ExposedByteArrayOutputStream"/> - <Method name="toByteArray"/> - <Bug pattern="EI_EXPOSE_REP"/> - <!--May expose internal representation by returning reference to mutable object--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.util.ExposedByteArrayOutputStream"/> - <Method name="writeAndOwn"/> - <Bug pattern="EI_EXPOSE_REP2"/> - <!--May expose internal representation by incorporating reference to mutable object--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.util.PCollectionViews$PCollectionViewBase"/> - <Method name="equals"/> - <Bug pattern="RCN_REDUNDANT_NULLCHECK_OF_NONNULL_VALUE"/> - <!--Redundant nullcheck of value known to be non-null--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.util.PCollectionViews$SingletonPCollectionView"/> - <Field name="defaultValue"/> - <Bug pattern="IS2_INCONSISTENT_SYNC"/> - <!--Inconsistent synchronization--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.util.PaneInfoTracker$1"/> - <Method name="readLater"/> - <Bug pattern="RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT"/> - <!--Return value of method without side effect is ignored--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.util.PubsubClient$IncomingMessage"/> - <Method name="<init>"/> - <Bug pattern="EI_EXPOSE_REP2"/> - <!--May expose internal representation by incorporating reference to mutable object--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.util.PubsubClient$IncomingMessage"/> - <Method name="<init>"/> - <Bug pattern="EI_EXPOSE_REP2"/> - <!--May expose internal representation by incorporating reference to mutable object--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.util.PubsubClient$OutgoingMessage"/> - <Method name="<init>"/> - <Bug pattern="EI_EXPOSE_REP2"/> - <!--May expose internal representation by incorporating reference to mutable object--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.util.ReleaseInfo"/> - <Method name="<init>"/> - <Bug pattern="OBL_UNSATISFIED_OBLIGATION"/> - <!--Method may fail to clean up stream or resource--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.util.SystemReduceFn"/> - <Method name="prefetchOnTrigger"/> - <Bug pattern="RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT"/> - <!--Return value of method without side effect is ignored--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.util.TriggerContextFactory"/> - <Field name="activeWindows"/> - <Bug pattern="URF_UNREAD_FIELD"/> - <!--Unread field--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.util.TriggerRunner"/> - <Method name="prefetchForMerge"/> - <Bug pattern="RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT"/> - <!--Return value of method without side effect is ignored--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.util.TriggerRunner"/> - <Method name="prefetchForValue"/> - <Bug pattern="RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT"/> - <!--Return value of method without side effect is ignored--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.util.TriggerRunner"/> - <Method name="prefetchOnFire"/> - <Bug pattern="RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT"/> - <!--Return value of method without side effect is ignored--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.util.TriggerRunner"/> - <Method name="prefetchShouldFire"/> - <Bug pattern="RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT"/> - <!--Return value of method without side effect is ignored--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.util.ValueWithRecordId"/> - <Method name="getId"/> - <Bug pattern="EI_EXPOSE_REP"/> - <!--May expose internal representation by returning reference to mutable object--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.util.ValueWithRecordId"/> - <Method name="<init>"/> - <Bug pattern="EI_EXPOSE_REP2"/> - <!--May expose internal representation by incorporating reference to mutable object--> + <!--[BEAM-419] Non-transient non-serializable instance field in serializable class--> </Match> <Match> <Class name="org.apache.beam.sdk.util.WatermarkHold"/> <Field name="timerInternals"/> <Bug pattern="SE_BAD_FIELD"/> - <!--Non-transient non-serializable instance field in serializable class--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.util.WatermarkHold$1"/> - <Method name="readLater"/> - <Bug pattern="RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT"/> - <!--Return value of method without side effect is ignored--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.util.common.ReflectHelpers$1"/> - <Method name="apply"/> - <Bug pattern="NP_PARAMETER_MUST_BE_NONNULL_BUT_MARKED_AS_NULLABLE"/> - <!--Parameter must be non-null but is marked as nullable--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.util.common.ReflectHelpers$2"/> - <Method name="apply"/> - <Bug pattern="NP_PARAMETER_MUST_BE_NONNULL_BUT_MARKED_AS_NULLABLE"/> - <!--Parameter must be non-null but is marked as nullable--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.util.common.ReflectHelpers$3"/> - <Method name="apply"/> - <Bug pattern="NP_PARAMETER_MUST_BE_NONNULL_BUT_MARKED_AS_NULLABLE"/> - <!--Parameter must be non-null but is marked as nullable--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.util.common.ReflectHelpers$4"/> - <Method name="apply"/> - <Bug pattern="NP_PARAMETER_MUST_BE_NONNULL_BUT_MARKED_AS_NULLABLE"/> - <!--Parameter must be non-null but is marked as nullable--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.util.common.ReflectHelpers$5"/> - <Method name="apply"/> - <Bug pattern="NP_PARAMETER_MUST_BE_NONNULL_BUT_MARKED_AS_NULLABLE"/> - <!--Parameter must be non-null but is marked as nullable--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.util.common.ReflectHelpers$6"/> - <Method name="apply"/> - <Bug pattern="NP_PARAMETER_MUST_BE_NONNULL_BUT_MARKED_AS_NULLABLE"/> - <!--Parameter must be non-null but is marked as nullable--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.util.state.StateMerging"/> - <Method name="mergeBags"/> - <Bug pattern="RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT"/> - <!--Return value of method without side effect is ignored--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.util.state.StateMerging"/> - <Method name="mergeCombiningValues"/> - <Bug pattern="RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT"/> - <!--Return value of method without side effect is ignored--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.util.state.StateMerging"/> - <Method name="prefetchBags"/> - <Bug pattern="RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT"/> - <!--Return value of method without side effect is ignored--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.util.state.StateMerging"/> - <Method name="prefetchCombiningValues"/> - <Bug pattern="RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT"/> - <!--Return value of method without side effect is ignored--> - </Match> - <Match> - <Class name="org.apache.beam.sdk.util.state.StateMerging"/> - <Method name="prefetchWatermarks"/> - <Bug pattern="RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT"/> - <!--Return value of method without side effect is ignored--> + <!--[BEAM-420] Non-transient non-serializable instance field in serializable class--> </Match> <Match> <Class name="org.apache.beam.sdk.util.state.StateTags$CombiningValueStateTag"/> <Method name="equals"/> <Bug pattern="EQ_DOESNT_OVERRIDE_EQUALS"/> - <!--Class doesn't override equals in superclass--> + <!--[BEAM-421] Class doesn't override equals in superclass--> </Match> </FindBugsFilter> http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3875071b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/InstantCoder.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/InstantCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/InstantCoder.java index 0be6ef8..693791c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/InstantCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/InstantCoder.java @@ -29,6 +29,8 @@ import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; + /** * A {@link Coder} for joda {@link Instant} that encodes it as a big endian {@link Long} * shifted such that lexicographic ordering of the bytes corresponds to chronological order. @@ -48,6 +50,9 @@ public class InstantCoder extends AtomicCoder<Instant> { private InstantCoder() {} + private static final Converter<Instant, Long> ORDER_PRESERVING_CONVERTER = + new LexicographicLongConverter(); + /** * Converts {@link Instant} to a {@code Long} representing its millis-since-epoch, * but shifted so that the byte representation of negative values are lexicographically @@ -56,19 +61,21 @@ public class InstantCoder extends AtomicCoder<Instant> { * <p>This deliberately utilizes the well-defined overflow for {@code Long} values. * See http://docs.oracle.com/javase/specs/jls/se7/html/jls-15.html#jls-15.18.2 */ - private static final Converter<Instant, Long> ORDER_PRESERVING_CONVERTER = - new Converter<Instant, Long>() { - - @Override - protected Long doForward(Instant instant) { - return instant.getMillis() - Long.MIN_VALUE; - } - - @Override - protected Instant doBackward(Long shiftedMillis) { - return new Instant(shiftedMillis + Long.MIN_VALUE); - } - }; + @SuppressFBWarnings(value = "HE_INHERITS_EQUALS_USE_HASHCODE", + justification = "Converter overrides .equals() to add documentation " + + "but does not change behavior") + private static class LexicographicLongConverter extends Converter<Instant, Long> { + + @Override + protected Long doForward(Instant instant) { + return instant.getMillis() - Long.MIN_VALUE; + } + + @Override + protected Instant doBackward(Long shiftedMillis) { + return new Instant(shiftedMillis + Long.MIN_VALUE); + } + } @Override public void encode(Instant value, OutputStream outStream, Context context) http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3875071b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java index c48933b..42d3c05 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java @@ -386,8 +386,7 @@ public class Write { // The WriteOperation's state is the same as after its initialization in the first do-once // ParDo. There is a dependency between this ParDo and the parallel write (the writer results // collection as a side input), so it will happen after the parallel write. - @SuppressWarnings("unused") - final PCollection<Integer> done = operationCollection + operationCollection .apply("Finalize", ParDo.of(new DoFn<WriteOperation<T, WriteT>, Integer>() { @Override public void processElement(ProcessContext c) throws Exception { http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3875071b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java index 3cb1377..f21b9b9 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java @@ -83,7 +83,7 @@ import java.util.SortedSet; import java.util.TreeMap; import java.util.TreeSet; -import javax.annotation.Nullable; +import javax.annotation.Nonnull; /** * Constructs a {@link PipelineOptions} or any derived interface that is composable to any other @@ -348,7 +348,7 @@ public class PipelineOptionsFactory { getRegisteredOptions(), new Predicate<Class<? extends PipelineOptions>>() { @Override - public boolean apply(Class<? extends PipelineOptions> input) { + public boolean apply(@Nonnull Class<? extends PipelineOptions> input) { if (helpOption.contains(".")) { return input.getName().endsWith(helpOption); } else { @@ -462,7 +462,7 @@ public class PipelineOptionsFactory { private static final Predicate<Method> NOT_SYNTHETIC_PREDICATE = new Predicate<Method>() { @Override - public boolean apply(Method input) { + public boolean apply(@Nonnull Method input) { return !input.isSynthetic(); } }; @@ -1388,7 +1388,7 @@ public class PipelineOptionsFactory { SortedSet<String> closestMatches = new TreeSet<String>( Sets.filter(propertyNamesToGetters.keySet(), new Predicate<String>() { @Override - public boolean apply(@Nullable String input) { + public boolean apply(@Nonnull String input) { return StringUtils.getLevenshteinDistance(entry.getKey(), input) <= 2; } })); @@ -1441,7 +1441,7 @@ public class PipelineOptionsFactory { List<String> values = FluentIterable.from(entry.getValue()) .transformAndConcat(new Function<String, Iterable<String>>() { @Override - public Iterable<String> apply(String input) { + public Iterable<String> apply(@Nonnull String input) { return Arrays.asList(input.split(",")); } }).toList(); http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3875071b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CoderProperties.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CoderProperties.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CoderProperties.java index ceed8c0..c217834 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CoderProperties.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CoderProperties.java @@ -38,6 +38,7 @@ import org.apache.beam.sdk.util.UnownedInputStream; import org.apache.beam.sdk.util.UnownedOutputStream; import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import java.io.ByteArrayInputStream; @@ -61,7 +62,7 @@ public class CoderProperties { /** * All the contexts, for use in test cases. */ - public static final List<Coder.Context> ALL_CONTEXTS = Arrays.asList( + public static final List<Coder.Context> ALL_CONTEXTS = ImmutableList.of( Coder.Context.OUTER, Coder.Context.NESTED); /** http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3875071b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java index 883b2b3..432d083 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java @@ -76,6 +76,8 @@ import java.util.List; import java.util.Map; import java.util.NoSuchElementException; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; + /** * An assertion on the contents of a {@link PCollection} incorporated into the pipeline. Such an * assertion can be checked no matter what kind of {@link PipelineRunner} is used. @@ -389,6 +391,7 @@ public class PAssert { */ @Deprecated @Override + @SuppressFBWarnings(value = "EQ_UNUSUAL", justification = "Unsupported operation") public boolean equals(Object o) { throw new UnsupportedOperationException( "If you meant to test object equality, use .containsInAnyOrder instead."); @@ -551,6 +554,7 @@ public class PAssert { */ @Deprecated @Override + @SuppressFBWarnings(value = "EQ_UNUSUAL", justification = "Unsupported operation") public boolean equals(Object o) { throw new UnsupportedOperationException( String.format( http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3875071b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SerializableMatchers.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SerializableMatchers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SerializableMatchers.java index 21173d1..45b0592 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SerializableMatchers.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SerializableMatchers.java @@ -38,6 +38,8 @@ import java.util.List; import javax.annotation.Nullable; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; + /** * Static class for building and using {@link SerializableMatcher} instances. * @@ -1148,6 +1150,8 @@ class SerializableMatchers implements Serializable { private static class SerializableArrayViaCoder<T> implements SerializableSupplier<T[]> { /** Cached value that is not serialized. */ @Nullable + @SuppressFBWarnings(value = "SE_TRANSIENT_FIELD_NOT_RESTORED", + justification = "Cached value is lazily restored on read.") private transient T[] value; /** The bytes of {@link #value} when encoded via {@link #coder}. */ http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3875071b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java index fb9e4b6..b3bcf9d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java @@ -160,7 +160,6 @@ public class DoFnTester<InputT, OutputT> { * {@code DoFn} doesn't emit to any side outputs. */ public void setSideOutputTags(TupleTagList sideOutputTags) { - this.sideOutputTags = sideOutputTags.getAll(); resetState(); } @@ -678,7 +677,6 @@ public class DoFnTester<InputT, OutputT> { /** The output tags used by the DoFn under test. */ TupleTag<OutputT> mainOutputTag = new TupleTag<>(); - List<TupleTag<?>> sideOutputTags = new ArrayList<>(); /** The original DoFn under test, if started. */ DoFn<InputT, OutputT> fn; http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3875071b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java index 6180dbf..ee7323b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java @@ -620,7 +620,7 @@ public class DisplayData implements Serializable { /** * Marker exception class for exceptions encountered while populating display data. */ - private class PopulateDisplayDataException extends RuntimeException { + private static class PopulateDisplayDataException extends RuntimeException { PopulateDisplayDataException(String message, Throwable cause) { super(message, cause); } http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3875071b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java index f4cf84d..5e4cb52 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java @@ -422,7 +422,7 @@ public class CoGbkResult { @Override public boolean hasNext() { - if (containsTag[tag] == Boolean.FALSE) { + if (Boolean.FALSE.equals(containsTag[tag])) { return false; } advance(); http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3875071b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterDelayFromFirstElement.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterDelayFromFirstElement.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterDelayFromFirstElement.java index 7ec3ce9..a0fc04d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterDelayFromFirstElement.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterDelayFromFirstElement.java @@ -44,6 +44,8 @@ import java.util.Locale; import java.util.Objects; import javax.annotation.Nullable; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; + /** * A base class for triggers that happen after a processing time delay from the arrival * of the first element in a pane. @@ -157,6 +159,8 @@ public abstract class AfterDelayFromFirstElement extends OnceTrigger { } @Override + @SuppressFBWarnings(value = "RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT", justification = + "prefetch side effect") public void prefetchOnElement(StateAccessor<?> state) { state.access(DELAYED_UNTIL_TAG).readLater(); } @@ -213,6 +217,8 @@ public abstract class AfterDelayFromFirstElement extends OnceTrigger { } @Override + @SuppressFBWarnings(value = "RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT", justification = + "prefetch side effect") public void prefetchShouldFire(StateAccessor<?> state) { state.access(DELAYED_UNTIL_TAG).readLater(); } http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3875071b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterPane.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterPane.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterPane.java index 977259d..7328987 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterPane.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterPane.java @@ -33,6 +33,8 @@ import org.joda.time.Instant; import java.util.List; import java.util.Objects; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; + /** * {@link Trigger}s that fire based on properties of the elements in the current pane. */ @@ -84,6 +86,8 @@ private static final StateTag<Object, AccumulatorCombiningState<Long, long[], Lo } @Override + @SuppressFBWarnings(value = "RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT", justification = + "prefetch side effect") public void prefetchShouldFire(StateAccessor<?> state) { state.access(ELEMENTS_IN_PANE_TAG).readLater(); } http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3875071b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOff.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOff.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOff.java index e26b7bd..df2897e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOff.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOff.java @@ -24,6 +24,8 @@ import com.google.api.client.util.NanoClock; import java.util.concurrent.TimeUnit; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; + /** * Extension of {@link AttemptBoundedExponentialBackOff} that bounds the total time that the backoff * is happening as well as the amount of retries. Acts exactly as a AttemptBoundedExponentialBackOff @@ -129,6 +131,8 @@ public class AttemptAndTimeBoundedExponentialBackOff extends AttemptBoundedExpon } @Override + @SuppressFBWarnings(value = "UR_UNINIT_READ_CALLED_FROM_SUPER_CONSTRUCTOR", + justification = "Explicitly handled in implementation.") public void reset() { // reset() is called in the constructor of the parent class before resetPolicy and nanoClock are // set. In this case, we call the parent class's reset() method and return. http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3875071b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExposedByteArrayInputStream.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExposedByteArrayInputStream.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExposedByteArrayInputStream.java index 9197fbe..eae3e40 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExposedByteArrayInputStream.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExposedByteArrayInputStream.java @@ -20,6 +20,8 @@ package org.apache.beam.sdk.util; import java.io.ByteArrayInputStream; import java.io.IOException; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; + /** * {@link ByteArrayInputStream} that allows accessing the entire internal buffer without copying. */ @@ -31,6 +33,7 @@ public class ExposedByteArrayInputStream extends ByteArrayInputStream{ /** Read all remaining bytes. * @throws IOException */ + @SuppressFBWarnings(value = "EI_EXPOSE_REP", justification = "Returns internal buffer by design") public byte[] readAll() throws IOException { if (pos == 0 && count == buf.length) { pos = count; http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3875071b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExposedByteArrayOutputStream.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExposedByteArrayOutputStream.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExposedByteArrayOutputStream.java index 412c8b6..48006ea 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExposedByteArrayOutputStream.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExposedByteArrayOutputStream.java @@ -20,6 +20,8 @@ package org.apache.beam.sdk.util; import java.io.ByteArrayOutputStream; import java.io.IOException; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; + /** * {@link ByteArrayOutputStream} special cased to treat writes of a single byte-array specially. * When calling {@link #toByteArray()} after writing only one {@code byte[]} using @@ -63,6 +65,7 @@ public class ExposedByteArrayOutputStream extends ByteArrayOutputStream { * * @throws IOException */ + @SuppressFBWarnings(value = "EI_EXPOSE_REP", justification = "Takes ownership of input buffer") public void writeAndOwn(byte[] b) throws IOException { if (b.length == 0) { return; @@ -92,6 +95,7 @@ public class ExposedByteArrayOutputStream extends ByteArrayOutputStream { } @Override + @SuppressFBWarnings(value = "EI_EXPOSE_REP", justification = "Returns internal buffer by design") public byte[] toByteArray() { // Note: count == buf.length is not a correct criteria to "return buf;", because the internal // buf may be reused after reset(). http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3875071b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PCollectionViews.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PCollectionViews.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PCollectionViews.java index 0b9107c..d63fb96 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PCollectionViews.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PCollectionViews.java @@ -165,8 +165,8 @@ public class PCollectionViews { throw new RuntimeException("Unexpected IOException: ", e); } } + return defaultValue; } - return defaultValue; } @Override @@ -412,7 +412,7 @@ public class PCollectionViews { @Override public boolean equals(Object other) { - if (!(other instanceof PCollectionView) || other == null) { + if (!(other instanceof PCollectionView)) { return false; } @SuppressWarnings("unchecked") http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3875071b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReleaseInfo.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReleaseInfo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReleaseInfo.java index 77289ac..ad03c79 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReleaseInfo.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReleaseInfo.java @@ -64,14 +64,12 @@ public final class ReleaseInfo extends GenericJson { private ReleaseInfo(String resourcePath) { Properties properties = new Properties(); - InputStream in = ReleaseInfo.class.getResourceAsStream( - PROPERTIES_PATH); - if (in == null) { - LOG.warn("Dataflow properties resource not found: {}", resourcePath); - return; - } + try (InputStream in = ReleaseInfo.class.getResourceAsStream(PROPERTIES_PATH)) { + if (in == null) { + LOG.warn("Dataflow properties resource not found: {}", resourcePath); + return; + } - try { properties.load(in); } catch (IOException e) { LOG.warn("Error loading Dataflow properties resource: ", e); http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3875071b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TriggerContextFactory.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TriggerContextFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TriggerContextFactory.java index f7635d3..03f1baa 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TriggerContextFactory.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TriggerContextFactory.java @@ -53,16 +53,13 @@ public class TriggerContextFactory<W extends BoundedWindow> { private final WindowFn<?, W> windowFn; private StateInternals<?> stateInternals; - // Future triggers may be able to exploit the active window to state address window mapping. - @SuppressWarnings("unused") - private ActiveWindowSet<W> activeWindows; private final Coder<W> windowCoder; public TriggerContextFactory(WindowFn<?, W> windowFn, StateInternals<?> stateInternals, ActiveWindowSet<W> activeWindows) { + // Future triggers may be able to exploit the active window to state address window mapping. this.windowFn = windowFn; this.stateInternals = stateInternals; - this.activeWindows = activeWindows; this.windowCoder = windowFn.windowCoder(); } http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3875071b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ReflectHelpers.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ReflectHelpers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ReflectHelpers.java index 2b6f6bc..75b8ad8 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ReflectHelpers.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ReflectHelpers.java @@ -38,6 +38,7 @@ import java.util.Collections; import java.util.LinkedHashSet; import java.util.Queue; +import javax.annotation.Nonnull; import javax.annotation.Nullable; /** @@ -50,7 +51,7 @@ public class ReflectHelpers { /** A {@link Function} that turns a method into a simple method signature. */ public static final Function<Method, String> METHOD_FORMATTER = new Function<Method, String>() { @Override - public String apply(Method input) { + public String apply(@Nonnull Method input) { String parameterTypes = FluentIterable.from(asList(input.getParameterTypes())) .transform(CLASS_SIMPLE_NAME) .join(COMMA_SEPARATOR); @@ -64,7 +65,7 @@ public class ReflectHelpers { public static final Function<Method, String> CLASS_AND_METHOD_FORMATTER = new Function<Method, String>() { @Override - public String apply(Method input) { + public String apply(@Nonnull Method input) { return String.format("%s#%s", CLASS_NAME.apply(input.getDeclaringClass()), METHOD_FORMATTER.apply(input)); @@ -75,7 +76,7 @@ public class ReflectHelpers { public static final Function<Class<?>, String> CLASS_NAME = new Function<Class<?>, String>() { @Override - public String apply(Class<?> input) { + public String apply(@Nonnull Class<?> input) { return input.getName(); } }; @@ -84,7 +85,7 @@ public class ReflectHelpers { public static final Function<Class<?>, String> CLASS_SIMPLE_NAME = new Function<Class<?>, String>() { @Override - public String apply(Class<?> input) { + public String apply(@Nonnull Class<?> input) { return input.getSimpleName(); } }; @@ -94,7 +95,7 @@ public class ReflectHelpers { new Function<Type, String>() { @Override @Nullable - public String apply(@Nullable Type input) { + public String apply(@Nonnull Type input) { StringBuilder builder = new StringBuilder(); format(builder, input); return builder.toString(); @@ -184,7 +185,7 @@ public class ReflectHelpers { return FluentIterable.from(interfaces).transformAndConcat( new Function<Class<?>, Iterable<Method>>() { @Override - public Iterable<Method> apply(Class<?> input) { + public Iterable<Method> apply(@Nonnull Class<?> input) { return getClosureOfMethodsOnInterface(input); } }); http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3875071b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateMerging.java ---------------------------------------------------------------------- diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateMerging.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateMerging.java index 843933f..c41a23e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateMerging.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateMerging.java @@ -28,6 +28,8 @@ import java.util.Collection; import java.util.List; import java.util.Map; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; + /** * Helpers for merging state. */ @@ -59,7 +61,7 @@ public class StateMerging { // Prefetch everything except what's already in result. for (BagState<T> source : map.values()) { if (!source.equals(result)) { - source.readLater(); + prefetchRead(source); } } } @@ -85,7 +87,7 @@ public class StateMerging { List<ReadableState<Iterable<T>>> futures = new ArrayList<>(sources.size()); for (BagState<T> source : sources) { if (!source.equals(result)) { - source.readLater(); + prefetchRead(source); futures.add(source); } } @@ -115,7 +117,7 @@ public class StateMerging { prefetchCombiningValues(MergingStateAccessor<K, W> context, StateTag<? super K, StateT> address) { for (StateT state : context.accessInEachMergingWindow(address).values()) { - state.readLater(); + prefetchRead(state); } } @@ -147,7 +149,7 @@ public class StateMerging { // Prefetch. List<ReadableState<AccumT>> futures = new ArrayList<>(sources.size()); for (AccumulatorCombiningState<InputT, AccumT, OutputT> source : sources) { - source.readLater(); + prefetchRead(source); } // Read. List<AccumT> accumulators = new ArrayList<>(futures.size()); @@ -188,10 +190,16 @@ public class StateMerging { } // Prefetch. for (WatermarkHoldState<W> source : map.values()) { - source.readLater(); + prefetchRead(source); } } + @SuppressFBWarnings(value = "RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT", + justification = "prefetch call readLater") + private static void prefetchRead(ReadableState<?> source) { + source.readLater(); + } + /** * Merge all watermark state in {@code address} across all merging windows in {@code context}, * where the final merge result window is {@code mergeResult}.
