diff --git a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/QueryTest.java b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/QueryTest.java index d4d51f17c71f..a3eba70898fc 100644 --- a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/QueryTest.java +++ b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/QueryTest.java @@ -25,6 +25,7 @@ import org.apache.beam.sdk.values.TimestampedValue; import org.junit.Ignore; +import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -33,23 +34,22 @@ * Test the various NEXMark queries yield results coherent with their models. */ @RunWith(JUnit4.class) -@Ignore -//TODO Ismael public class QueryTest { private static final NexmarkConfiguration CONFIG = NexmarkConfiguration.DEFAULT.clone(); + @Rule + public TestPipeline p = TestPipeline.create(); static { - CONFIG.numEvents = 2000; + CONFIG.numEvents = 100; } /** Test {@code query} matches {@code model}. */ - private static void queryMatchesModel(String name, NexmarkQuery query, NexmarkQueryModel model) { - Pipeline p = TestPipeline.create(); + private void queryMatchesModel(String name, NexmarkQuery query, NexmarkQueryModel model) { NexmarkUtils.setupPipeline(NexmarkUtils.CoderStrategy.HAND, p); PCollection> results = p.apply(name + ".ReadBounded", NexmarkUtils.batchEventsSource(CONFIG)).apply(query); //TODO Ismael this should not be called explicitly -// results.setIsBoundedInternal(IsBounded.BOUNDED); + results.setIsBoundedInternal(PCollection.IsBounded.BOUNDED); PAssert.that(results).satisfies(model.assertionFor()); p.run().waitUntilFinish(); }