Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
The table of contents is too big for display.
Diff view
Diff view
  •  
  •  
  •  
Original file line number Diff line number Diff line change
Expand Up @@ -94,12 +94,6 @@ class BeamModulePlugin implements Plugin<Project> {
/** Classes triggering Checker failures. A map from class name to the bug filed against checkerframework. */
Map<String, String> classesTriggerCheckerBugs = [:]

/**
Some module's tests take a very long time to compile with checkerframework.
Until that is solved, set this flag to skip checking for those tests.
*/
boolean checkerTooSlowOnTests = false

/** Controls whether the dependency analysis plugin is enabled. */
boolean enableStrictDependencies = false

Expand Down Expand Up @@ -833,7 +827,12 @@ class BeamModulePlugin implements Plugin<Project> {
'org.checkerframework.checker.nullness.NullnessChecker'
]

excludeTests = configuration.checkerTooSlowOnTests
// Always exclude checkerframework on tests. It's slow, and it often
// raises erroneous error because we don't have checker annotations for
// test libraries like junit and hamcrest. See BEAM-11436.
// Consider re-enabling if we can get annotations for the test libraries
// we use.
excludeTests = true

extraJavacArgs = [
"-AskipDefs=${skipDefCombinedRegex}",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -62,9 +62,6 @@

/** End-to-end integration test of {@link WindowedWordCount}. */
@RunWith(JUnit4.class)
@SuppressWarnings({
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
public class WindowedWordCountIT {

@Rule public TestName testName = new TestName();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,9 +42,6 @@

/** End-to-end tests of TrafficMaxLaneFlowIT. */
@RunWith(JUnit4.class)
@SuppressWarnings({
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
public class TrafficMaxLaneFlowIT {
private static final Logger LOG = LoggerFactory.getLogger(TrafficMaxLaneFlowIT.class);
private TrafficMaxLaneFlowOptions options;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,9 +42,6 @@

/** End-to-end tests of TrafficRoutes. */
@RunWith(JUnit4.class)
@SuppressWarnings({
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
public class TrafficRoutesIT {
private static final Logger LOG = LoggerFactory.getLogger(TrafficRoutesIT.class);
private TrafficRoutesOptions options;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -68,7 +68,6 @@
/** Tests for {@link CoderTranslation}. */
@SuppressWarnings({
"rawtypes", // TODO(https://issues.apache.org/jira/browse/BEAM-10556)
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
public class CoderTranslationTest {
private static final Set<Coder<?>> KNOWN_CODERS =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -91,7 +91,6 @@
@RunWith(Parameterized.class)
@SuppressWarnings({
"rawtypes", // TODO(https://issues.apache.org/jira/browse/BEAM-10556)
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
public class CommonCoderTest {
private static final String STANDARD_CODERS_YAML_PATH =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,9 +30,6 @@
import org.junit.runners.JUnit4;

@RunWith(JUnit4.class)
@SuppressWarnings({
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
public class DefaultArtifactResolverTest {
private RunnerApi.Pipeline createEmptyPipeline(
Iterable<RunnerApi.ArtifactInformation> dependencies) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,9 +47,6 @@

/** Tests for {@link ForwardingPTransform}. */
@RunWith(JUnit4.class)
@SuppressWarnings({
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
public class ForwardingPTransformTest {
@Rule public ExpectedException thrown = ExpectedException.none();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,6 @@
@RunWith(JUnit4.class)
@SuppressWarnings({
"rawtypes", // TODO(https://issues.apache.org/jira/browse/BEAM-10556)
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
public class MorePipelineTest {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -63,9 +63,6 @@

/** Tests for {@link PCollectionTranslation}. */
@RunWith(Parameterized.class)
@SuppressWarnings({
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
public class PCollectionTranslationTest {
// Each spec activates tests of all subsets of its fields
@Parameters(name = "{index}: {0}")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -85,7 +85,6 @@
@RunWith(JUnit4.class)
@SuppressWarnings({
"rawtypes", // TODO(https://issues.apache.org/jira/browse/BEAM-10556)
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
public class PTransformMatchersTest implements Serializable {
@Rule
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,9 +41,6 @@

/** Tests for {@link PTransformReplacements}. */
@RunWith(JUnit4.class)
@SuppressWarnings({
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
public class PTransformReplacementsTest {
@Rule public TestPipeline pipeline = TestPipeline.create().enableAbandonedNodeEnforcement(false);
@Rule public ExpectedException thrown = ExpectedException.none();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -62,9 +62,6 @@

/** Tests for {@link PTransformTranslation}. */
@RunWith(Parameterized.class)
@SuppressWarnings({
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
public class PTransformTranslationTest {

@Parameters(name = "{index}: {0}")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,9 +42,6 @@
import org.junit.runners.Parameterized.Parameters;

/** Tests for {@link PipelineOptionsTranslation}. */
@SuppressWarnings({
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
public class PipelineOptionsTranslationTest {
/** Tests that translations can round-trip through the proto format. */
@RunWith(Parameterized.class)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -72,7 +72,6 @@
@RunWith(Parameterized.class)
@SuppressWarnings({
"rawtypes", // TODO(https://issues.apache.org/jira/browse/BEAM-10556)
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
public class PipelineTranslationTest {
@Parameter(0)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,9 +45,6 @@

/** Tests for {@link ReplacementOutputs}. */
@RunWith(JUnit4.class)
@SuppressWarnings({
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
public class ReplacementOutputsTest {
@Rule public ExpectedException thrown = ExpectedException.none();
private TestPipeline p = TestPipeline.create();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,9 +42,6 @@

/** Tests for {@link SchemaTranslation}. */
@RunWith(Enclosed.class)
@SuppressWarnings({
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
public class SchemaTranslationTest {

/** Tests round-trip proto encodings for {@link Schema}. */
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,9 +50,6 @@

/** Tests for {@link SdkComponents}. */
@RunWith(JUnit4.class)
@SuppressWarnings({
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
public class SdkComponentsTest {
@Rule public TestPipeline pipeline = TestPipeline.create().enableAbandonedNodeEnforcement(false);
@Rule public ExpectedException thrown = ExpectedException.none();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,9 +34,6 @@

/** Tests for {@link Timer}. */
@RunWith(JUnit4.class)
@SuppressWarnings({
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
public class TimerTest {
private static final Instant FIRE_TIME = new Instant(123L);
private static final Instant HOLD_TIME = new Instant(456L);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,9 +43,6 @@

/** Tests for utilities in {@link TriggerTranslation}. */
@RunWith(Parameterized.class)
@SuppressWarnings({
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
public class TriggerTranslationTest {

@AutoValue
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -72,9 +72,6 @@

/** Unit tests for {@link UnboundedReadFromBoundedSource}. */
@RunWith(JUnit4.class)
@SuppressWarnings({
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
public class UnboundedReadFromBoundedSourceTest {

@Rule public TemporaryFolder tmpFolder = new TemporaryFolder();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,6 @@
@RunWith(JUnit4.class)
@SuppressWarnings({
"rawtypes", // TODO(https://issues.apache.org/jira/browse/BEAM-10556)
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
public class UnsupportedOverrideFactoryTest {
@Rule public ExpectedException thrown = ExpectedException.none();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -85,9 +85,6 @@
* details.
*/
@RunWith(JUnit4.class)
@SuppressWarnings({
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
public class ValidateRunnerXlangTest implements Serializable {
@Rule public transient TestPipeline testPipeline = TestPipeline.create();
private PipelineResult pipelineResult;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -64,7 +64,6 @@
@RunWith(Enclosed.class)
@SuppressWarnings({
"rawtypes", // TODO(https://issues.apache.org/jira/browse/BEAM-10556)
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
public class WindowingStrategyTranslationTest {
// Each spec activates tests of all subsets of its fields
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -60,9 +60,6 @@

/** Tests for {@link GreedyPipelineFuser}. */
@RunWith(JUnit4.class)
@SuppressWarnings({
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
public class GreedyPipelineFuserTest {
// Contains the 'go' and 'py' environments, and a default 'impulse' step and output.
private Components partialComponents;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -54,9 +54,6 @@

/** Tests for {@link GreedyStageFuser}. */
@RunWith(JUnit4.class)
@SuppressWarnings({
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
public class GreedyStageFuserTest {
@Rule public ExpectedException thrown = ExpectedException.none();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,6 @@
@RunWith(JUnit4.class)
@SuppressWarnings({
"rawtypes", // TODO(https://issues.apache.org/jira/browse/BEAM-10556)
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
public class OutputDeduplicatorTest {
@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,9 +40,6 @@
import org.junit.rules.TemporaryFolder;
import org.mockito.Mockito;

@SuppressWarnings({
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
public class ClasspathScanningResourcesDetectorTest {

@Rule public transient TemporaryFolder tmpFolder = new TemporaryFolder();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,9 +41,6 @@

/** Tests for PipelineResources. */
@RunWith(JUnit4.class)
@SuppressWarnings({
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
public class PipelineResourcesTest {

@Rule public transient TemporaryFolder tmpFolder = new TemporaryFolder();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,9 +30,6 @@

/** Tests for {@link InMemoryTimerInternals}. */
@RunWith(JUnit4.class)
@SuppressWarnings({
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
public class InMemoryTimerInternalsTest {

private static final StateNamespace NS1 = new StateNamespaceForTest("NS1");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,9 +44,6 @@

/** Unit tests for {@link LateDataDroppingDoFnRunner}. */
@RunWith(JUnit4.class)
@SuppressWarnings({
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
public class LateDataDroppingDoFnRunnerTest {
private static final FixedWindows WINDOW_FN = FixedWindows.of(Duration.millis(10));

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,6 @@
/** Tests for {@link OutputAndTimeBoundedSplittableProcessElementInvoker}. */
@SuppressWarnings({
"rawtypes", // TODO(https://issues.apache.org/jira/browse/BEAM-10556)
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
public class OutputAndTimeBoundedSplittableProcessElementInvokerTest {
@Rule public transient ExpectedException e = ExpectedException.none();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -85,7 +85,6 @@
*/
@SuppressWarnings({
"rawtypes", // TODO(https://issues.apache.org/jira/browse/BEAM-10556)
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
public class ReduceFnTester<InputT, OutputT, W extends BoundedWindow> {
private static final String KEY = "TEST_KEY";
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,9 +44,6 @@

/** Unit tests for {@link SideInputHandler}. */
@RunWith(JUnit4.class)
@SuppressWarnings({
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
public class SideInputHandlerTest {

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -63,7 +63,6 @@
@RunWith(JUnit4.class)
@SuppressWarnings({
"rawtypes", // TODO(https://issues.apache.org/jira/browse/BEAM-10556)
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
public class SimpleDoFnRunnerTest {
@Rule public ExpectedException thrown = ExpectedException.none();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -79,7 +79,6 @@
@RunWith(JUnit4.class)
@SuppressWarnings({
"rawtypes", // TODO(https://issues.apache.org/jira/browse/BEAM-10556)
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
public class SimplePushbackSideInputDoFnRunnerTest {
@Mock StepContext mockStepContext;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -77,9 +77,6 @@

/** Tests for {@link SplittableParDoViaKeyedWorkItems.ProcessFn}. */
@RunWith(JUnit4.class)
@SuppressWarnings({
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
public class SplittableParDoProcessFnTest {
private static final int MAX_OUTPUTS_PER_BUNDLE = 10000;
private static final Duration MAX_BUNDLE_DURATION = Duration.standardSeconds(5);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -60,9 +60,6 @@
import org.junit.Test;

/** Tests for {@link StateInternals}. */
@SuppressWarnings({
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
public abstract class StateInternalsTest {

private static final BoundedWindow WINDOW_1 = new IntervalWindow(new Instant(0), new Instant(10));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -63,7 +63,6 @@
@RunWith(JUnit4.class)
@SuppressWarnings({
"rawtypes", // TODO(https://issues.apache.org/jira/browse/BEAM-10556)
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
public class StatefulDoFnRunnerTest {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,9 +31,6 @@
import org.joda.time.Instant;

/** Matchers that are useful for working with Windowing, Timestamps, etc. */
@SuppressWarnings({
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
public class WindowMatchers {

public static <T> Matcher<WindowedValue<? extends T>> isWindowedValue(
Expand Down
Loading