diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index d9f3de5ea766..a72b1ae2607b 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -1554,7 +1554,6 @@ class BeamModulePlugin implements Plugin { "InvalidThrows", "JavaTimeDefaultTimeZone", "JavaUtilDate", - "JodaConstructors", "MalformedInlineTag", "MissingSummary", "MixedMutabilityReturnType", diff --git a/examples/java/src/main/java/org/apache/beam/examples/KafkaStreaming.java b/examples/java/src/main/java/org/apache/beam/examples/KafkaStreaming.java index f0b09226865a..4626c2828e97 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/KafkaStreaming.java +++ b/examples/java/src/main/java/org/apache/beam/examples/KafkaStreaming.java @@ -115,7 +115,7 @@ public static void main(String[] args) { // starts. Duration windowSize = Duration.standardSeconds(WINDOW_TIME); Instant nextWindowStart = - new Instant( + Instant.ofEpochMilli( Instant.now().getMillis() + windowSize.getMillis() - Instant.now().plus(windowSize).getMillis() % windowSize.getMillis()); @@ -156,7 +156,8 @@ public void run() { .apply( GenerateSequence.from(0) .withRate(MESSAGES_COUNT, Duration.standardSeconds(WINDOW_TIME)) - .withTimestampFn((Long n) -> new Instant(System.currentTimeMillis()))) + .withTimestampFn( + (Long n) -> Instant.ofEpochMilli(System.currentTimeMillis()))) .apply(ParDo.of(new RandomUserScoreGeneratorFn())); input.apply( KafkaIO.write() diff --git a/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java index 3d67f28e9f10..0c5fa754ddef 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/WindowedWordCount.java @@ -126,7 +126,7 @@ static class AddTimestampFn extends DoFn { @ProcessElement public void processElement(@Element String element, OutputReceiver receiver) { Instant randomTimestamp = - new Instant( + Instant.ofEpochMilli( ThreadLocalRandom.current() .nextLong(minTimestamp.getMillis(), maxTimestamp.getMillis())); @@ -189,8 +189,8 @@ public interface Options static void runWindowedWordCount(Options options) throws IOException { final String output = options.getOutput(); - final Instant minTimestamp = new Instant(options.getMinTimestampMillis()); - final Instant maxTimestamp = new Instant(options.getMaxTimestampMillis()); + final Instant minTimestamp = Instant.ofEpochMilli(options.getMinTimestampMillis()); + final Instant maxTimestamp = Instant.ofEpochMilli(options.getMaxTimestampMillis()); Pipeline pipeline = Pipeline.create(options); diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java index b06cd8da9d43..4dccf825a619 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java @@ -105,7 +105,7 @@ public void processElement(ProcessContext c) { String userName = (String) row.get("contributor_username"); if (userName != null) { // Sets the implicit timestamp field to be used in windowing. - c.outputWithTimestamp(userName, new Instant(timestamp * 1000L)); + c.outputWithTimestamp(userName, Instant.ofEpochMilli(timestamp * 1000L)); } } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java index 6f75e2e03d99..3b558272f2bd 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java @@ -188,7 +188,8 @@ public void processElement(DoFn.ProcessContext c) throws Excepti if (items.length > 0) { try { String timestamp = items[0]; - c.outputWithTimestamp(c.element(), new Instant(dateTimeFormat.parseMillis(timestamp))); + c.outputWithTimestamp( + c.element(), Instant.ofEpochMilli(dateTimeFormat.parseMillis(timestamp))); } catch (IllegalArgumentException e) { // Skip the invalid input. } diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java index f28cd2d3e6a4..7c71fe7a86c3 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java @@ -192,7 +192,8 @@ public void processElement(DoFn.ProcessContext c) throws Excepti String timestamp = tryParseTimestamp(items); if (timestamp != null) { try { - c.outputWithTimestamp(c.element(), new Instant(dateTimeFormat.parseMillis(timestamp))); + c.outputWithTimestamp( + c.element(), Instant.ofEpochMilli(dateTimeFormat.parseMillis(timestamp))); } catch (IllegalArgumentException e) { // Skip the invalid input. } diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java b/examples/java/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java index c57d9ba6b8c8..b6ac4a3c85aa 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java @@ -130,8 +130,9 @@ public static void main(String[] args) throws Exception { Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); Pipeline pipeline = Pipeline.create(options); - final Instant stopMinTimestamp = new Instant(minFmt.parseMillis(options.getStopMin())); - final Instant startMinTimestamp = new Instant(minFmt.parseMillis(options.getStartMin())); + final Instant stopMinTimestamp = Instant.ofEpochMilli(minFmt.parseMillis(options.getStopMin())); + final Instant startMinTimestamp = + Instant.ofEpochMilli(minFmt.parseMillis(options.getStartMin())); // Read 'gaming' events from a text file. pipeline @@ -161,7 +162,7 @@ public static void main(String[] args) throws Exception { // Add an element timestamp based on the event log, and apply fixed windowing. .apply( "AddEventTimestamps", - WithTimestamps.of((GameActionInfo i) -> new Instant(i.getTimestamp()))) + WithTimestamps.of((GameActionInfo i) -> Instant.ofEpochMilli(i.getTimestamp()))) .apply( "FixedWindowsTeam", Window.into(FixedWindows.of(Duration.standardMinutes(options.getWindowDuration())))) diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/TriggerExample.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/TriggerExample.java index cd3c6dd84157..2ec061669b4b 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/TriggerExample.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/TriggerExample.java @@ -515,7 +515,7 @@ public void processElement(ProcessContext c) throws Exception { int range = MAX_DELAY - MIN_DELAY; int delayInMinutes = random.nextInt(range) + MIN_DELAY; long delayInMillis = TimeUnit.MINUTES.toMillis(delayInMinutes); - timestamp = new Instant(timestamp.getMillis() - delayInMillis); + timestamp = Instant.ofEpochMilli(timestamp.getMillis() - delayInMillis); } c.outputWithTimestamp(c.element(), timestamp); } diff --git a/examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java b/examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java index 4f24c69f74b7..7c3ef36191e3 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java +++ b/examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java @@ -802,21 +802,21 @@ public static void main(String[] args) { Create.timestamped( TimestampedValue.of( new TableRow().set("user", "mobile").set("score", 12).set("gap", 5), - new Instant()), + Instant.now()), TimestampedValue.of( - new TableRow().set("user", "desktop").set("score", 4), new Instant()), + new TableRow().set("user", "desktop").set("score", 4), Instant.now()), TimestampedValue.of( new TableRow().set("user", "mobile").set("score", -3).set("gap", 5), - new Instant().plus(Duration.millis(2000))), + Instant.now().plus(Duration.millis(2000))), TimestampedValue.of( new TableRow().set("user", "mobile").set("score", 2).set("gap", 5), - new Instant().plus(Duration.millis(9000))), + Instant.now().plus(Duration.millis(9000))), TimestampedValue.of( new TableRow().set("user", "mobile").set("score", 7).set("gap", 5), - new Instant().plus(Duration.millis(12000))), + Instant.now().plus(Duration.millis(12000))), TimestampedValue.of( new TableRow().set("user", "desktop").set("score", 10), - new Instant().plus(Duration.millis(12000))))); + Instant.now().plus(Duration.millis(12000))))); // [END CustomSessionWindow5] // [START CustomSessionWindow6] diff --git a/examples/java/src/test/java/org/apache/beam/examples/WindowedWordCountIT.java b/examples/java/src/test/java/org/apache/beam/examples/WindowedWordCountIT.java index fa25d72cb22a..075dfae89030 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/WindowedWordCountIT.java +++ b/examples/java/src/test/java/org/apache/beam/examples/WindowedWordCountIT.java @@ -158,7 +158,8 @@ private void testWindowedWordCountPipeline(WindowedWordCountITOptions options) t for (int startMinute : ImmutableList.of(0, 10, 20, 30, 40, 50)) { final Instant windowStart = - new Instant(options.getMinTimestampMillis()).plus(Duration.standardMinutes(startMinute)); + Instant.ofEpochMilli(options.getMinTimestampMillis()) + .plus(Duration.standardMinutes(startMinute)); String filePrefix = filenamePolicy.filenamePrefixForWindow( new IntervalWindow(windowStart, windowStart.plus(Duration.standardMinutes(10)))); diff --git a/examples/java/src/test/java/org/apache/beam/examples/complete/AutoCompleteTest.java b/examples/java/src/test/java/org/apache/beam/examples/complete/AutoCompleteTest.java index 59a21aa8eb39..faac8b326090 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/complete/AutoCompleteTest.java +++ b/examples/java/src/test/java/org/apache/beam/examples/complete/AutoCompleteTest.java @@ -111,11 +111,11 @@ public void testTinyAutoComplete() { public void testWindowedAutoComplete() { List> words = Arrays.asList( - TimestampedValue.of("xA", new Instant(1)), - TimestampedValue.of("xA", new Instant(1)), - TimestampedValue.of("xB", new Instant(1)), - TimestampedValue.of("xB", new Instant(2)), - TimestampedValue.of("xB", new Instant(2))); + TimestampedValue.of("xA", Instant.ofEpochMilli(1)), + TimestampedValue.of("xA", Instant.ofEpochMilli(1)), + TimestampedValue.of("xB", Instant.ofEpochMilli(1)), + TimestampedValue.of("xB", Instant.ofEpochMilli(2)), + TimestampedValue.of("xB", Instant.ofEpochMilli(2))); PCollection input = p.apply(Create.timestamped(words)); diff --git a/examples/java/src/test/java/org/apache/beam/examples/complete/game/HourlyTeamScoreTest.java b/examples/java/src/test/java/org/apache/beam/examples/complete/game/HourlyTeamScoreTest.java index 46d7b41746ab..f5da37e699db 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/complete/game/HourlyTeamScoreTest.java +++ b/examples/java/src/test/java/org/apache/beam/examples/complete/game/HourlyTeamScoreTest.java @@ -90,7 +90,7 @@ public class HourlyTeamScoreTest implements Serializable { @Test public void testUserScoresFilter() throws Exception { - final Instant startMinTimestamp = new Instant(1447965680000L); + final Instant startMinTimestamp = Instant.ofEpochMilli(1447965680000L); PCollection input = p.apply(Create.of(GAME_EVENTS).withCoder(StringUtf8Coder.of())); diff --git a/examples/java/src/test/java/org/apache/beam/examples/complete/game/LeaderBoardTest.java b/examples/java/src/test/java/org/apache/beam/examples/complete/game/LeaderBoardTest.java index 13f80012cc1b..4394e4de80b7 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/complete/game/LeaderBoardTest.java +++ b/examples/java/src/test/java/org/apache/beam/examples/complete/game/LeaderBoardTest.java @@ -49,7 +49,7 @@ public class LeaderBoardTest implements Serializable { private static final Duration ALLOWED_LATENESS = Duration.standardHours(1); private static final Duration TEAM_WINDOW_DURATION = Duration.standardMinutes(20); - private Instant baseTime = new Instant(0); + private Instant baseTime = Instant.ofEpochMilli(0); @Rule public TestPipeline p = TestPipeline.create(); /** Some example users, on two separate teams. */ diff --git a/examples/java/src/test/java/org/apache/beam/examples/complete/game/StatefulTeamScoreTest.java b/examples/java/src/test/java/org/apache/beam/examples/complete/game/StatefulTeamScoreTest.java index d94d74d2a85b..aa9cb7796646 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/complete/game/StatefulTeamScoreTest.java +++ b/examples/java/src/test/java/org/apache/beam/examples/complete/game/StatefulTeamScoreTest.java @@ -44,7 +44,7 @@ @RunWith(JUnit4.class) public class StatefulTeamScoreTest { - private Instant baseTime = new Instant(0); + private Instant baseTime = Instant.ofEpochMilli(0); @Rule public TestPipeline p = TestPipeline.create(); diff --git a/examples/java/src/test/java/org/apache/beam/examples/cookbook/TriggerExampleTest.java b/examples/java/src/test/java/org/apache/beam/examples/cookbook/TriggerExampleTest.java index 19c83c6eb73c..ae2e842e7c19 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/cookbook/TriggerExampleTest.java +++ b/examples/java/src/test/java/org/apache/beam/examples/cookbook/TriggerExampleTest.java @@ -65,17 +65,17 @@ public class TriggerExampleTest { "01/01/2010 00:00:00,1108302,5,W,ML,36,100,30,0.0065,66,9,1,0.001," + "74.8,1,9,3,0.0028,71,1,9,12,0.0099,87.4,1,9,13,0.0121,99.0,1,,,,,0,,,,,0,,,,,0,,," + ",,0", - new Instant(60000)), + Instant.ofEpochMilli(60000)), TimestampedValue.of( "01/01/2010 00:00:00,1108302,110,E,ML,36,100,40,0.0065,66,9,1,0.001," + "74.8,1,9,3,0.0028,71,1,9,12,0.0099,67.4,1,9,13,0.0121,99.0,1,,,,,0,,,,,0,,,,,0,,," + ",,0", - new Instant(1)), + Instant.ofEpochMilli(1)), TimestampedValue.of( "01/01/2010 00:00:00,1108302,110,E,ML,36,100,50,0.0065,66,9,1," + "0.001,74.8,1,9,3,0.0028,71,1,9,12,0.0099,97.4,1,9,13,0.0121,50.0,1,,,,,0,,,,,0" + ",,,,,0,,,,,0", - new Instant(1))); + Instant.ofEpochMilli(1))); private static final TableRow OUT_ROW_1 = new TableRow() diff --git a/examples/java/twitter/src/main/java/org/apache/beam/examples/twitterstreamgenerator/ReadFromTwitterDoFn.java b/examples/java/twitter/src/main/java/org/apache/beam/examples/twitterstreamgenerator/ReadFromTwitterDoFn.java index 64852bac81a9..e0f557b29cb8 100644 --- a/examples/java/twitter/src/main/java/org/apache/beam/examples/twitterstreamgenerator/ReadFromTwitterDoFn.java +++ b/examples/java/twitter/src/main/java/org/apache/beam/examples/twitterstreamgenerator/ReadFromTwitterDoFn.java @@ -45,7 +45,7 @@ final class ReadFromTwitterDoFn extends DoFn { private final DateTime startTime; ReadFromTwitterDoFn() { - this.startTime = new DateTime(); + this.startTime = DateTime.now(); } /* Logger for class.*/ private static final Logger LOG = LoggerFactory.getLogger(ReadFromTwitterDoFn.class); diff --git a/model/fn-execution/src/main/resources/org/apache/beam/model/fnexecution/v1/standard_coders.yaml b/model/fn-execution/src/main/resources/org/apache/beam/model/fnexecution/v1/standard_coders.yaml index 8ce9d2751545..2d7f87825d2d 100644 --- a/model/fn-execution/src/main/resources/org/apache/beam/model/fnexecution/v1/standard_coders.yaml +++ b/model/fn-execution/src/main/resources/org/apache/beam/model/fnexecution/v1/standard_coders.yaml @@ -30,7 +30,7 @@ # # Java code snippet to generate example bytes: # Coder> coder = Timer.Coder.of(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE); -# Instant now = new Instant(1000L); +# Instant now = Instant.ofEpochMilli(1000L); # Timer timer = Timer.of( # "key", # "tag", @@ -597,7 +597,7 @@ examples: # Java code snippet to generate example bytes: # TimestampPrefixingWindowCoder coder = TimestampPrefixingWindowCoder.of(IntervalWindowCoder.of()); -# Instant end = new Instant(-9223372036854410L); +# Instant end = Instant.ofEpochMilli(-9223372036854410L); # Duration span = Duration.millis(365L); # IntervalWindow window = new IntervalWindow(end.minus(span), span); # byte[] bytes = CoderUtils.encodeToByteArray(coder, window); diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/WindowMatchers.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/WindowMatchers.java index 243dbc139ee3..b3c47f2634c7 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/WindowMatchers.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/WindowMatchers.java @@ -123,10 +123,10 @@ public static Matcher> isSingleWindowedValue( public static Matcher> isSingleWindowedValue( Matcher valueMatcher, long timestamp, long windowStart, long windowEnd) { IntervalWindow intervalWindow = - new IntervalWindow(new Instant(windowStart), new Instant(windowEnd)); + new IntervalWindow(Instant.ofEpochMilli(windowStart), Instant.ofEpochMilli(windowEnd)); return WindowMatchers.isSingleWindowedValue( valueMatcher, - Matchers.describedAs("%0", Matchers.equalTo(new Instant(timestamp)), timestamp), + Matchers.describedAs("%0", Matchers.equalTo(Instant.ofEpochMilli(timestamp)), timestamp), Matchers.equalTo(intervalWindow), Matchers.anything()); } @@ -138,10 +138,10 @@ public static Matcher> isSingleWindowedValue( long windowEnd, PaneInfo paneInfo) { IntervalWindow intervalWindow = - new IntervalWindow(new Instant(windowStart), new Instant(windowEnd)); + new IntervalWindow(Instant.ofEpochMilli(windowStart), Instant.ofEpochMilli(windowEnd)); return WindowMatchers.isSingleWindowedValue( valueMatcher, - Matchers.describedAs("%0", Matchers.equalTo(new Instant(timestamp)), timestamp), + Matchers.describedAs("%0", Matchers.equalTo(Instant.ofEpochMilli(timestamp)), timestamp), Matchers.equalTo(intervalWindow), Matchers.equalTo(paneInfo)); } @@ -186,7 +186,8 @@ public static Matcher> isSingleWindowedValue( } public static Matcher intervalWindow(long start, long end) { - return Matchers.equalTo(new IntervalWindow(new Instant(start), new Instant(end))); + return Matchers.equalTo( + new IntervalWindow(Instant.ofEpochMilli(start), Instant.ofEpochMilli(end))); } public static Matcher> valueWithPaneInfo(final PaneInfo paneInfo) { diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MonitoringInfoEncodings.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MonitoringInfoEncodings.java index 7b1b3f698935..19266435dd7d 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MonitoringInfoEncodings.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MonitoringInfoEncodings.java @@ -101,7 +101,7 @@ public static ByteString encodeInt64Gauge(GaugeData data) { public static GaugeData decodeInt64Gauge(ByteString payload) { InputStream input = payload.newInput(); try { - Instant timestamp = new Instant(VARINT_CODER.decode(input)); + Instant timestamp = Instant.ofEpochMilli(VARINT_CODER.decode(input)); return GaugeData.create(VARINT_CODER.decode(input), timestamp); } catch (IOException e) { throw new RuntimeException(e); diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterDelayFromFirstElementStateMachine.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterDelayFromFirstElementStateMachine.java index e8ca0990559b..4b08dfd5fb0c 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterDelayFromFirstElementStateMachine.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterDelayFromFirstElementStateMachine.java @@ -121,7 +121,7 @@ public AfterDelayFromFirstElementStateMachine alignedTo( * the epoch. */ public AfterDelayFromFirstElementStateMachine alignedTo(final Duration size) { - return alignedTo(size, new Instant(0)); + return alignedTo(size, Instant.ofEpochMilli(0)); } /** diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachines.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachines.java index 32c915333f7b..913c6e46d134 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachines.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachines.java @@ -95,7 +95,7 @@ private static TriggerStateMachine stateMachineForAfterProcessingTime( stateMachine = stateMachine.alignedTo( Duration.millis(transform.getAlignTo().getPeriod()), - new Instant(transform.getAlignTo().getOffset())); + Instant.ofEpochMilli(transform.getAlignTo().getOffset())); break; case DELAY: stateMachine = diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryBundleFinalizerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryBundleFinalizerTest.java index 4b96e5005515..b669fb535841 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryBundleFinalizerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryBundleFinalizerTest.java @@ -35,7 +35,7 @@ public void testCallbackRegistration() { InMemoryBundleFinalizer finalizer = new InMemoryBundleFinalizer(); // Check when nothing has been registered assertThat(finalizer.getAndClearFinalizations(), is(empty())); - finalizer.afterBundleCommit(new Instant(), () -> {}); + finalizer.afterBundleCommit(Instant.now(), () -> {}); assertThat(finalizer.getAndClearFinalizations(), hasSize(1)); // Check that it is empty assertThat(finalizer.getAndClearFinalizations(), is(empty())); diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryTimerInternalsTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryTimerInternalsTest.java index 3cc7a248fda2..6bb37ff1ef29 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryTimerInternalsTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryTimerInternalsTest.java @@ -45,28 +45,28 @@ public void testFiringEventTimers() throws Exception { TimerData.of( ID1, NS1, - new Instant(19), - new Instant(19), + Instant.ofEpochMilli(19), + Instant.ofEpochMilli(19), TimeDomain.EVENT_TIME, CausedByDrain.NORMAL); TimerData eventTimer2 = TimerData.of( ID2, NS1, - new Instant(29), - new Instant(29), + Instant.ofEpochMilli(29), + Instant.ofEpochMilli(29), TimeDomain.EVENT_TIME, CausedByDrain.NORMAL); underTest.setTimer(eventTimer1); underTest.setTimer(eventTimer2); - underTest.advanceInputWatermark(new Instant(20)); + underTest.advanceInputWatermark(Instant.ofEpochMilli(20)); assertThat(underTest.removeNextEventTimer(), equalTo(eventTimer1)); assertThat(underTest.removeNextEventTimer(), nullValue()); // Advancing just a little shouldn't refire - underTest.advanceInputWatermark(new Instant(21)); + underTest.advanceInputWatermark(Instant.ofEpochMilli(21)); assertThat(underTest.removeNextEventTimer(), nullValue()); // Adding the timer and advancing a little should refire @@ -75,7 +75,7 @@ public void testFiringEventTimers() throws Exception { assertThat(underTest.removeNextEventTimer(), nullValue()); // And advancing the rest of the way should still have the other timer - underTest.advanceInputWatermark(new Instant(30)); + underTest.advanceInputWatermark(Instant.ofEpochMilli(30)); assertThat(underTest.removeNextEventTimer(), equalTo(eventTimer2)); assertThat(underTest.removeNextEventTimer(), nullValue()); } @@ -83,10 +83,10 @@ public void testFiringEventTimers() throws Exception { @Test public void testResetById() throws Exception { InMemoryTimerInternals underTest = new InMemoryTimerInternals(); - Instant earlyTimestamp = new Instant(13); - Instant laterTimestamp = new Instant(42); + Instant earlyTimestamp = Instant.ofEpochMilli(13); + Instant laterTimestamp = Instant.ofEpochMilli(42); - underTest.advanceInputWatermark(new Instant(0)); + underTest.advanceInputWatermark(Instant.ofEpochMilli(0)); underTest.setTimer(NS1, ID1, "", earlyTimestamp, earlyTimestamp, TimeDomain.EVENT_TIME); underTest.setTimer(NS1, ID1, "", laterTimestamp, laterTimestamp, TimeDomain.EVENT_TIME); underTest.advanceInputWatermark(earlyTimestamp.plus(Duration.millis(1L))); @@ -101,7 +101,7 @@ public void testResetById() throws Exception { @Test public void testDeletionIdempotent() throws Exception { InMemoryTimerInternals underTest = new InMemoryTimerInternals(); - Instant timestamp = new Instant(42); + Instant timestamp = Instant.ofEpochMilli(42); underTest.setTimer(NS1, ID1, ID1, timestamp, timestamp, TimeDomain.EVENT_TIME); underTest.deleteTimer(NS1, ID1, ID1); underTest.deleteTimer(NS1, ID1, ID1); @@ -110,12 +110,12 @@ public void testDeletionIdempotent() throws Exception { @Test public void testDeletionById() throws Exception { InMemoryTimerInternals underTest = new InMemoryTimerInternals(); - Instant timestamp = new Instant(42); + Instant timestamp = Instant.ofEpochMilli(42); - underTest.advanceInputWatermark(new Instant(0)); + underTest.advanceInputWatermark(Instant.ofEpochMilli(0)); underTest.setTimer(NS1, ID1, ID1, timestamp, timestamp, TimeDomain.EVENT_TIME); underTest.deleteTimer(NS1, ID1, ID1); - underTest.advanceInputWatermark(new Instant(43)); + underTest.advanceInputWatermark(Instant.ofEpochMilli(43)); assertThat(underTest.removeNextEventTimer(), nullValue()); } @@ -126,37 +126,37 @@ public void testFiringProcessingTimeTimers() throws Exception { TimerData processingTime1 = TimerData.of( NS1, - new Instant(19), - new Instant(19), + Instant.ofEpochMilli(19), + Instant.ofEpochMilli(19), TimeDomain.PROCESSING_TIME, CausedByDrain.NORMAL); TimerData processingTime2 = TimerData.of( NS1, - new Instant(29), - new Instant(29), + Instant.ofEpochMilli(29), + Instant.ofEpochMilli(29), TimeDomain.PROCESSING_TIME, CausedByDrain.NORMAL); underTest.setTimer(processingTime1); underTest.setTimer(processingTime2); - underTest.advanceProcessingTime(new Instant(20)); + underTest.advanceProcessingTime(Instant.ofEpochMilli(20)); assertThat(underTest.removeNextProcessingTimer(), equalTo(processingTime1)); assertThat(underTest.removeNextProcessingTimer(), nullValue()); // Advancing just a little shouldn't refire - underTest.advanceProcessingTime(new Instant(21)); + underTest.advanceProcessingTime(Instant.ofEpochMilli(21)); assertThat(underTest.removeNextProcessingTimer(), nullValue()); // Adding the timer and advancing a little should fire again underTest.setTimer(processingTime1); - underTest.advanceProcessingTime(new Instant(21)); + underTest.advanceProcessingTime(Instant.ofEpochMilli(21)); assertThat(underTest.removeNextProcessingTimer(), equalTo(processingTime1)); assertThat(underTest.removeNextProcessingTimer(), nullValue()); // And advancing the rest of the way should still have the other timer - underTest.advanceProcessingTime(new Instant(30)); + underTest.advanceProcessingTime(Instant.ofEpochMilli(30)); assertThat(underTest.removeNextProcessingTimer(), equalTo(processingTime2)); assertThat(underTest.removeNextProcessingTimer(), nullValue()); } @@ -166,36 +166,44 @@ public void testTimerOrdering() throws Exception { InMemoryTimerInternals underTest = new InMemoryTimerInternals(); TimerData eventTime1 = TimerData.of( - NS1, new Instant(19), new Instant(19), TimeDomain.EVENT_TIME, CausedByDrain.NORMAL); + NS1, + Instant.ofEpochMilli(19), + Instant.ofEpochMilli(19), + TimeDomain.EVENT_TIME, + CausedByDrain.NORMAL); TimerData processingTime1 = TimerData.of( NS1, - new Instant(19), - new Instant(19), + Instant.ofEpochMilli(19), + Instant.ofEpochMilli(19), TimeDomain.PROCESSING_TIME, CausedByDrain.NORMAL); TimerData synchronizedProcessingTime1 = TimerData.of( NS1, - new Instant(19), - new Instant(19), + Instant.ofEpochMilli(19), + Instant.ofEpochMilli(19), TimeDomain.SYNCHRONIZED_PROCESSING_TIME, CausedByDrain.NORMAL); TimerData eventTime2 = TimerData.of( - NS1, new Instant(29), new Instant(29), TimeDomain.EVENT_TIME, CausedByDrain.NORMAL); + NS1, + Instant.ofEpochMilli(29), + Instant.ofEpochMilli(29), + TimeDomain.EVENT_TIME, + CausedByDrain.NORMAL); TimerData processingTime2 = TimerData.of( NS1, - new Instant(29), - new Instant(29), + Instant.ofEpochMilli(29), + Instant.ofEpochMilli(29), TimeDomain.PROCESSING_TIME, CausedByDrain.NORMAL); TimerData synchronizedProcessingTime2 = TimerData.of( NS1, - new Instant(29), - new Instant(29), + Instant.ofEpochMilli(29), + Instant.ofEpochMilli(29), TimeDomain.SYNCHRONIZED_PROCESSING_TIME, CausedByDrain.NORMAL); @@ -207,19 +215,19 @@ public void testTimerOrdering() throws Exception { underTest.setTimer(synchronizedProcessingTime2); assertThat(underTest.removeNextEventTimer(), nullValue()); - underTest.advanceInputWatermark(new Instant(30)); + underTest.advanceInputWatermark(Instant.ofEpochMilli(30)); assertThat(underTest.removeNextEventTimer(), equalTo(eventTime1)); assertThat(underTest.removeNextEventTimer(), equalTo(eventTime2)); assertThat(underTest.removeNextEventTimer(), nullValue()); assertThat(underTest.removeNextProcessingTimer(), nullValue()); - underTest.advanceProcessingTime(new Instant(30)); + underTest.advanceProcessingTime(Instant.ofEpochMilli(30)); assertThat(underTest.removeNextProcessingTimer(), equalTo(processingTime1)); assertThat(underTest.removeNextProcessingTimer(), equalTo(processingTime2)); assertThat(underTest.removeNextProcessingTimer(), nullValue()); assertThat(underTest.removeNextSynchronizedProcessingTimer(), nullValue()); - underTest.advanceSynchronizedProcessingTime(new Instant(30)); + underTest.advanceSynchronizedProcessingTime(Instant.ofEpochMilli(30)); assertThat( underTest.removeNextSynchronizedProcessingTimer(), equalTo(synchronizedProcessingTime1)); assertThat( @@ -232,20 +240,24 @@ public void testDeduplicate() throws Exception { InMemoryTimerInternals underTest = new InMemoryTimerInternals(); TimerData eventTime = TimerData.of( - NS1, new Instant(19), new Instant(19), TimeDomain.EVENT_TIME, CausedByDrain.NORMAL); + NS1, + Instant.ofEpochMilli(19), + Instant.ofEpochMilli(19), + TimeDomain.EVENT_TIME, + CausedByDrain.NORMAL); TimerData processingTime = TimerData.of( NS1, - new Instant(19), - new Instant(19), + Instant.ofEpochMilli(19), + Instant.ofEpochMilli(19), TimeDomain.PROCESSING_TIME, CausedByDrain.NORMAL); underTest.setTimer(eventTime); underTest.setTimer(eventTime); underTest.setTimer(processingTime); underTest.setTimer(processingTime); - underTest.advanceProcessingTime(new Instant(20)); - underTest.advanceInputWatermark(new Instant(20)); + underTest.advanceProcessingTime(Instant.ofEpochMilli(20)); + underTest.advanceInputWatermark(Instant.ofEpochMilli(20)); assertThat(underTest.removeNextProcessingTimer(), equalTo(processingTime)); assertThat(underTest.removeNextProcessingTimer(), nullValue()); diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/KeyedWorkItemCoderTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/KeyedWorkItemCoderTest.java index 56f6248d747a..c4be9699711d 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/KeyedWorkItemCoderTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/KeyedWorkItemCoderTest.java @@ -47,8 +47,8 @@ public void testEncodeDecodeEqual() throws Exception { ImmutableList.of( TimerData.of( StateNamespaces.global(), - new Instant(500L), - new Instant(500L), + Instant.ofEpochMilli(500L), + Instant.ofEpochMilli(500L), TimeDomain.EVENT_TIME, CausedByDrain.NORMAL)); Iterable> elements = diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunnerTest.java index 5a6257ba0a1b..3e5843029b24 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunnerTest.java @@ -59,7 +59,7 @@ public void setUp() { public void testLateDataFilter() throws Exception { MetricsContainerImpl container = new MetricsContainerImpl("any"); MetricsEnvironment.setCurrentContainer(container); - when(mockTimerInternals.currentInputWatermarkTime()).thenReturn(new Instant(15L)); + when(mockTimerInternals.currentInputWatermarkTime()).thenReturn(Instant.ofEpochMilli(15L)); LateDataFilter lateDataFilter = new LateDataFilter(WindowingStrategy.of(WINDOW_FN), mockTimerInternals); @@ -97,7 +97,7 @@ public void testLateDataFilter() throws Exception { } private WindowedValue createDatum(T element, long timestampMillis) { - Instant timestamp = new Instant(timestampMillis); + Instant timestamp = Instant.ofEpochMilli(timestampMillis); return WindowedValues.of( element, timestamp, Arrays.asList(WINDOW_FN.assignWindow(timestamp)), PaneInfo.NO_FIRING); } diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/LateDataUtilsTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/LateDataUtilsTest.java index 4118890b89cd..7d2dedc2f553 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/LateDataUtilsTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/LateDataUtilsTest.java @@ -44,7 +44,7 @@ public void beforeEndOfGlobalWindowSame() { .withWindowFn(windowFn) .withAllowedLateness(allowedLateness); - IntervalWindow window = windowFn.assignWindow(new Instant(10)); + IntervalWindow window = windowFn.assignWindow(Instant.ofEpochMilli(10)); assertThat( LateDataUtils.garbageCollectionTime(window, strategy), equalTo(window.maxTimestamp().plus(allowedLateness))); @@ -71,7 +71,7 @@ public void garbageCollectionTimeAfterEndOfGlobalWindowWithLateness() { .withWindowFn(windowFn) .withAllowedLateness(allowedLateness); - IntervalWindow window = windowFn.assignWindow(new Instant(-100)); + IntervalWindow window = windowFn.assignWindow(Instant.ofEpochMilli(-100)); assertThat( window.maxTimestamp().plus(allowedLateness), Matchers.greaterThan(GlobalWindow.INSTANCE.maxTimestamp())); diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/MergingActiveWindowSetTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/MergingActiveWindowSetTest.java index c822f3c77d42..2248eaad4fbf 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/MergingActiveWindowSetTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/MergingActiveWindowSetTest.java @@ -77,7 +77,7 @@ public Object element() { @Override public Instant timestamp() { - return new Instant(instant); + return Instant.ofEpochMilli(instant); } @Override @@ -139,7 +139,7 @@ private void cleanup() { } private IntervalWindow window(long start, long size) { - return new IntervalWindow(new Instant(start), Duration.millis(size)); + return new IntervalWindow(Instant.ofEpochMilli(start), Duration.millis(size)); } @Test diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java index 85f6573be23e..dc58357d9d73 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java @@ -130,13 +130,13 @@ public void setUp() { PCollectionView mockViewUnchecked = mock(PCollectionView.class, withSettings().serializable()); mockView = mockViewUnchecked; - firstWindow = new IntervalWindow(new Instant(0), new Instant(10)); + firstWindow = new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10)); } private void injectElement(ReduceFnTester tester, int element) throws Exception { doNothing().when(mockTriggerStateMachine).onElement(anyElementContext()); - tester.injectElements(TimestampedValue.of(element, new Instant(element))); + tester.injectElements(TimestampedValue.of(element, Instant.ofEpochMilli(element))); } private void injectElements( @@ -145,7 +145,7 @@ private void injectElements( doNothing().when(mockTriggerStateMachine).onElement(anyElementContext()); List> timestampedValues = new ArrayList<>(); for (int value : values) { - timestampedValues.add(TimestampedValue.of(value, new Instant(value))); + timestampedValues.add(TimestampedValue.of(value, Instant.ofEpochMilli(value))); } tester.injectElements(timestampedValues); } @@ -183,13 +183,14 @@ public void testProcessingTimeTimerDoesNotGc() throws Exception { ReduceFnTester tester = ReduceFnTester.combining(strategy, Sum.ofIntegers(), VarIntCoder.of()); - tester.advanceProcessingTime(new Instant(5000)); + tester.advanceProcessingTime(Instant.ofEpochMilli(5000)); injectElement(tester, 2); // processing timer @ 5000 + 10; EOW timer @ 100 injectElement(tester, 5); - tester.advanceProcessingTime(new Instant(10000)); + tester.advanceProcessingTime(Instant.ofEpochMilli(10000)); - tester.assertHasOnlyGlobalAndStateFor(new IntervalWindow(new Instant(0), new Instant(100))); + tester.assertHasOnlyGlobalAndStateFor( + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(100))); assertThat( tester.extractOutput(), @@ -289,9 +290,9 @@ public void testSessionEowAndGcTogether() throws Exception { tester.setAutoAdvanceOutputWatermark(true); - tester.advanceInputWatermark(new Instant(0)); + tester.advanceInputWatermark(Instant.ofEpochMilli(0)); injectElement(tester, 1); - tester.advanceInputWatermark(new Instant(100)); + tester.advanceInputWatermark(Instant.ofEpochMilli(100)); assertThat( tester.extractOutput(), @@ -316,9 +317,9 @@ public void testFixedWindowsEowAndGcTogether() throws Exception { tester.setAutoAdvanceOutputWatermark(true); - tester.advanceInputWatermark(new Instant(0)); + tester.advanceInputWatermark(Instant.ofEpochMilli(0)); injectElement(tester, 1); - tester.advanceInputWatermark(new Instant(100)); + tester.advanceInputWatermark(Instant.ofEpochMilli(100)); assertThat( tester.extractOutput(), @@ -343,9 +344,9 @@ public void testFixedWindowsEowAndGcTogetherFireIfNonEmpty() throws Exception { tester.setAutoAdvanceOutputWatermark(true); - tester.advanceInputWatermark(new Instant(0)); + tester.advanceInputWatermark(Instant.ofEpochMilli(0)); injectElement(tester, 1); - tester.advanceInputWatermark(new Instant(100)); + tester.advanceInputWatermark(Instant.ofEpochMilli(100)); List>> output = tester.extractOutput(); assertThat( @@ -370,20 +371,20 @@ public void testOnlyOneOnTimePane() throws Exception { ReduceFnTester tester = ReduceFnTester.combining(strategy, Sum.ofIntegers(), VarIntCoder.of()); - tester.advanceInputWatermark(new Instant(0)); + tester.advanceInputWatermark(Instant.ofEpochMilli(0)); int value1 = 1; int value2 = 3; // A single element that should be in the ON_TIME output - tester.injectElements(TimestampedValue.of(value1, new Instant(1))); + tester.injectElements(TimestampedValue.of(value1, Instant.ofEpochMilli(1))); // Should fire ON_TIME - tester.advanceInputWatermark(new Instant(10)); + tester.advanceInputWatermark(Instant.ofEpochMilli(10)); // The DefaultTrigger should cause output labeled LATE, even though it does not have to be // labeled as such. - tester.injectElements(TimestampedValue.of(value2, new Instant(3))); + tester.injectElements(TimestampedValue.of(value2, Instant.ofEpochMilli(3))); List> output = tester.extractOutput(); assertEquals(2, output.size()); @@ -450,16 +451,16 @@ public void testLateProcessingTimeTimer() throws Exception { ReduceFnTester tester = ReduceFnTester.combining(strategy, Sum.ofIntegers(), VarIntCoder.of()); - tester.advanceProcessingTime(new Instant(5000)); + tester.advanceProcessingTime(Instant.ofEpochMilli(5000)); injectElement(tester, 2); // processing timer @ 5000 + 10; EOW timer @ 100 injectElement(tester, 5); // After this advancement, the window is expired and only the GC process // should be allowed to touch it - tester.advanceInputWatermarkNoTimers(new Instant(100)); + tester.advanceInputWatermarkNoTimers(Instant.ofEpochMilli(100)); // This should not output - tester.advanceProcessingTime(new Instant(6000)); + tester.advanceProcessingTime(Instant.ofEpochMilli(6000)); assertThat(tester.extractOutput(), emptyIterable()); } @@ -482,18 +483,18 @@ public void testCombiningAccumulatingProcessingTime() throws Exception { ReduceFnTester tester = ReduceFnTester.combining(strategy, Sum.ofIntegers(), VarIntCoder.of()); - tester.advanceProcessingTime(new Instant(5000)); + tester.advanceProcessingTime(Instant.ofEpochMilli(5000)); injectElement(tester, 2); // processing timer @ 5000 + 10; EOW timer @ 100 injectElement(tester, 5); - tester.advanceInputWatermarkNoTimers(new Instant(100)); - tester.advanceProcessingTimeNoTimers(new Instant(5010)); + tester.advanceInputWatermarkNoTimers(Instant.ofEpochMilli(100)); + tester.advanceProcessingTimeNoTimers(Instant.ofEpochMilli(5010)); // Fires the GC/EOW timer at the same time as the processing time timer. tester.fireTimers( - new IntervalWindow(new Instant(0), new Instant(100)), - TimestampedValue.of(TimeDomain.EVENT_TIME, new Instant(100)), - TimestampedValue.of(TimeDomain.PROCESSING_TIME, new Instant(5010))); + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(100)), + TimestampedValue.of(TimeDomain.EVENT_TIME, Instant.ofEpochMilli(100)), + TimestampedValue.of(TimeDomain.PROCESSING_TIME, Instant.ofEpochMilli(5010))); assertThat( tester.extractOutput(), @@ -586,12 +587,12 @@ public void testCombiningAccumulatingProcessingTimeSeparateBundles() throws Exce ReduceFnTester tester = ReduceFnTester.combining(strategy, Sum.ofIntegers(), VarIntCoder.of()); - tester.advanceProcessingTime(new Instant(5000)); + tester.advanceProcessingTime(Instant.ofEpochMilli(5000)); injectElement(tester, 2); // processing timer @ 5000 + 10; EOW timer @ 100 injectElement(tester, 5); - tester.advanceInputWatermark(new Instant(100)); - tester.advanceProcessingTime(new Instant(5011)); + tester.advanceInputWatermark(Instant.ofEpochMilli(100)); + tester.advanceProcessingTime(Instant.ofEpochMilli(5011)); assertThat( tester.extractOutput(), @@ -619,7 +620,7 @@ public void testCombiningAccumulatingEventTime() throws Exception { injectElement(tester, 2); // processing timer @ 5000 + 10; EOW timer @ 100 injectElement(tester, 5); - tester.advanceInputWatermark(new Instant(1000)); + tester.advanceInputWatermark(Instant.ofEpochMilli(1000)); assertThat( tester.extractOutput(), @@ -744,10 +745,11 @@ public void testWatermarkHoldAndLateData() throws Exception { assertEquals(null, tester.getOutputWatermark()); // All on time data, verify watermark hold. - IntervalWindow expectedWindow = new IntervalWindow(new Instant(0), new Instant(10)); + IntervalWindow expectedWindow = + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10)); injectElement(tester, 1); injectElement(tester, 3); - assertEquals(new Instant(1), tester.getWatermarkHold()); + assertEquals(Instant.ofEpochMilli(1), tester.getWatermarkHold()); when(mockTriggerStateMachine.shouldFire(anyTriggerContext())).thenReturn(true); injectElement(tester, 2); List>> output = tester.extractOutput(); @@ -756,7 +758,7 @@ public void testWatermarkHoldAndLateData() throws Exception { contains( isSingleWindowedValue( containsInAnyOrder(1, 2, 3), - equalTo(new Instant(1)), + equalTo(Instant.ofEpochMilli(1)), equalTo((BoundedWindow) expectedWindow)))); assertThat( output.get(0).getPaneInfo(), @@ -774,12 +776,12 @@ public void testWatermarkHoldAndLateData() throws Exception { assertEquals(0, droppedElements); // Input watermark -> 4, output watermark should advance that far as well - tester.advanceInputWatermark(new Instant(4)); - assertEquals(new Instant(4), tester.getOutputWatermark()); + tester.advanceInputWatermark(Instant.ofEpochMilli(4)); + assertEquals(Instant.ofEpochMilli(4), tester.getOutputWatermark()); // Some late, some on time. Verify that we only hold to the minimum of on-time. when(mockTriggerStateMachine.shouldFire(anyTriggerContext())).thenReturn(false); - tester.advanceInputWatermark(new Instant(4)); + tester.advanceInputWatermark(Instant.ofEpochMilli(4)); injectElement(tester, 2); injectElement(tester, 3); @@ -790,7 +792,7 @@ public void testWatermarkHoldAndLateData() throws Exception { // Now data just ahead of the output watermark arrives and sets an earlier "element" hold injectElement(tester, 5); - assertEquals(new Instant(5), tester.getWatermarkHold()); + assertEquals(Instant.ofEpochMilli(5), tester.getWatermarkHold()); when(mockTriggerStateMachine.shouldFire(anyTriggerContext())).thenReturn(true); injectElement(tester, 4); @@ -815,7 +817,7 @@ public void testWatermarkHoldAndLateData() throws Exception { // All behind the output watermark -- hold is at GC time (if we imagine the // trigger sets a timer for ON_TIME firing, that is actually when they'll be emitted) when(mockTriggerStateMachine.shouldFire(anyTriggerContext())).thenReturn(false); - tester.advanceInputWatermark(new Instant(8)); + tester.advanceInputWatermark(Instant.ofEpochMilli(8)); injectElement(tester, 6); injectElement(tester, 5); assertThat( @@ -865,7 +867,7 @@ public void testWatermarkHoldAndLateData() throws Exception { assertEquals(0, droppedElements); // Exceed the GC limit, triggering the last pane to be fired - tester.advanceInputWatermark(new Instant(50)); + tester.advanceInputWatermark(Instant.ofEpochMilli(50)); output = tester.extractOutput(); // Output time is still end of the window, because the new data (8) was behind // the output watermark. @@ -883,13 +885,13 @@ public void testWatermarkHoldAndLateData() throws Exception { 10))); // window end assertThat( output.get(0).getPaneInfo(), equalTo(PaneInfo.createPane(false, true, Timing.LATE, 3, 1))); - assertEquals(new Instant(50), tester.getOutputWatermark()); + assertEquals(Instant.ofEpochMilli(50), tester.getOutputWatermark()); assertEquals(null, tester.getWatermarkHold()); // Late timers are ignored tester.fireTimer( - new IntervalWindow(new Instant(0), new Instant(10)), - new Instant(12), + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10)), + Instant.ofEpochMilli(12), TimeDomain.EVENT_TIME); // And because we're past the end of window + allowed lateness, everything should be cleaned up. @@ -914,7 +916,7 @@ public void testWatermarkHoldForLateNewWindow() throws Exception { assertEquals(null, tester.getWatermarkHold()); assertEquals(null, tester.getOutputWatermark()); - tester.advanceInputWatermark(new Instant(40)); + tester.advanceInputWatermark(Instant.ofEpochMilli(40)); injectElements(tester, 1); assertThat(tester.getWatermarkHold(), nullValue()); injectElements(tester, 10); @@ -938,7 +940,7 @@ public void testMergingWatermarkHoldLateNewWindowMerged() throws Exception { assertEquals(null, tester.getWatermarkHold()); assertEquals(null, tester.getOutputWatermark()); - tester.advanceInputWatermark(new Instant(24)); + tester.advanceInputWatermark(Instant.ofEpochMilli(24)); injectElements(tester, 1); assertThat(tester.getWatermarkHold(), nullValue()); injectElements(tester, 14); @@ -946,23 +948,23 @@ public void testMergingWatermarkHoldLateNewWindowMerged() throws Exception { injectElements(tester, 6, 16); // There should now be a watermark hold since the window has extended past the input watermark. // The hold should be for the end of the window (last element + gapDuration - 1). - assertEquals(tester.getWatermarkHold(), new Instant(25)); + assertEquals(tester.getWatermarkHold(), Instant.ofEpochMilli(25)); injectElements(tester, 6, 21); // The hold should be extended with the window. - assertEquals(tester.getWatermarkHold(), new Instant(30)); + assertEquals(tester.getWatermarkHold(), Instant.ofEpochMilli(30)); // Advancing the watermark should remove the hold. - tester.advanceInputWatermark(new Instant(31)); + tester.advanceInputWatermark(Instant.ofEpochMilli(31)); assertThat(tester.getWatermarkHold(), nullValue()); // Late elements added to the window should not generate a hold. injectElements(tester, 0); assertThat(tester.getWatermarkHold(), nullValue()); // Generate a new window that is ontime. injectElements(tester, 32, 40); - assertEquals(tester.getWatermarkHold(), new Instant(49)); + assertEquals(tester.getWatermarkHold(), Instant.ofEpochMilli(49)); // Join the closed window with the new window. injectElements(tester, 24); - assertEquals(tester.getWatermarkHold(), new Instant(49)); - tester.advanceInputWatermark(new Instant(50)); + assertEquals(tester.getWatermarkHold(), Instant.ofEpochMilli(49)); + tester.advanceInputWatermark(Instant.ofEpochMilli(50)); assertThat(tester.getWatermarkHold(), nullValue()); } @@ -987,16 +989,16 @@ public void testMergingLateWatermarkHolds() throws Exception { assertEquals(null, tester.getWatermarkHold()); assertEquals(null, tester.getOutputWatermark()); - tester.advanceInputWatermark(new Instant(20)); + tester.advanceInputWatermark(Instant.ofEpochMilli(20)); // Add two late elements that cause a window to merge. injectElements(tester, Arrays.asList(3)); assertThat(tester.getWatermarkHold(), nullValue()); injectElements(tester, Arrays.asList(4)); - Instant endOfWindow = new Instant(4).plus(gapDuration); + Instant endOfWindow = Instant.ofEpochMilli(4).plus(gapDuration); // We expect a GC hold to be one less than the end of window plus the allowed lateness. Instant expectedGcHold = endOfWindow.plus(allowedLateness).minus(Duration.millis(1)); assertEquals(expectedGcHold, tester.getWatermarkHold()); - tester.advanceInputWatermark(new Instant(1000)); + tester.advanceInputWatermark(Instant.ofEpochMilli(1000)); assertEquals(expectedGcHold, tester.getWatermarkHold()); } @@ -1056,22 +1058,24 @@ public void testMergingWatermarkHoldAndLateDataFuzz() throws Exception { LOG.info("nextWatermark {} {}", nextWatermark, enabled); watermark = nextWatermark; tester.setAutoAdvanceOutputWatermark(enabled); - tester.advanceInputWatermark(new Instant(watermark)); + tester.advanceInputWatermark(Instant.ofEpochMilli(watermark)); } } split = nextSplit; Instant hold = tester.getWatermarkHold(); if (hold != null) { - assertThat(hold, greaterThanOrEqualTo(new Instant(watermark))); + assertThat(hold, greaterThanOrEqualTo(Instant.ofEpochMilli(watermark))); assertThat(watermark, lessThan(maxTs + gapDuration.getMillis())); } } tester.setAutoAdvanceOutputWatermark(true); watermark = gapDuration.getMillis() + maxTs; - tester.advanceInputWatermark(new Instant(watermark)); + tester.advanceInputWatermark(Instant.ofEpochMilli(watermark)); LOG.info("Output {}", tester.extractOutput()); if (tester.getWatermarkHold() != null) { - assertThat(tester.getWatermarkHold(), equalTo(new Instant(watermark).plus(allowedLateness))); + assertThat( + tester.getWatermarkHold(), + equalTo(Instant.ofEpochMilli(watermark).plus(allowedLateness))); } // Nothing dropped. long droppedElements = @@ -1096,35 +1100,38 @@ public void dontSetHoldIfTooLateForEndOfWindowTimer() throws Exception { tester.setAutoAdvanceOutputWatermark(false); // Case: Unobservably "late" relative to input watermark, but on time for output watermark - tester.advanceInputWatermark(new Instant(15)); - tester.advanceOutputWatermark(new Instant(11)); + tester.advanceInputWatermark(Instant.ofEpochMilli(15)); + tester.advanceOutputWatermark(Instant.ofEpochMilli(11)); - IntervalWindow expectedWindow = new IntervalWindow(new Instant(10), new Instant(20)); + IntervalWindow expectedWindow = + new IntervalWindow(Instant.ofEpochMilli(10), Instant.ofEpochMilli(20)); injectElement(tester, 14); // Hold was applied, waiting for end-of-window timer. - assertEquals(new Instant(14), tester.getWatermarkHold()); + assertEquals(Instant.ofEpochMilli(14), tester.getWatermarkHold()); // Trigger the end-of-window timer, fire a timer as though the mock trigger set it when(mockTriggerStateMachine.shouldFire(anyTriggerContext())).thenReturn(true); - tester.advanceInputWatermark(new Instant(20)); + tester.advanceInputWatermark(Instant.ofEpochMilli(20)); tester.fireTimer(expectedWindow, expectedWindow.maxTimestamp(), TimeDomain.EVENT_TIME); when(mockTriggerStateMachine.shouldFire(anyTriggerContext())).thenReturn(false); // Hold has been replaced with garbage collection hold. Waiting for garbage collection. - assertEquals(new Instant(29), tester.getWatermarkHold()); - assertEquals(new Instant(29), tester.getNextTimer(TimeDomain.EVENT_TIME)); + assertEquals(Instant.ofEpochMilli(29), tester.getWatermarkHold()); + assertEquals(Instant.ofEpochMilli(29), tester.getNextTimer(TimeDomain.EVENT_TIME)); // Case: Maybe late 1 injectElement(tester, 13); // No change to hold or timers. - assertEquals(new Instant(29), tester.getWatermarkHold()); - assertEquals(new Instant(29), tester.getNextTimer(TimeDomain.EVENT_TIME)); + assertEquals(Instant.ofEpochMilli(29), tester.getWatermarkHold()); + assertEquals(Instant.ofEpochMilli(29), tester.getNextTimer(TimeDomain.EVENT_TIME)); // Trigger the garbage collection timer. - tester.advanceInputWatermark(new Instant(30)); + tester.advanceInputWatermark(Instant.ofEpochMilli(30)); // Everything should be cleaned up. - assertFalse(tester.isMarkedFinished(new IntervalWindow(new Instant(10), new Instant(20)))); + assertFalse( + tester.isMarkedFinished( + new IntervalWindow(Instant.ofEpochMilli(10), Instant.ofEpochMilli(20)))); tester.assertHasOnlyGlobalAndFinishedSetsFor(); } @@ -1138,7 +1145,7 @@ public void testPaneInfoAllStates() throws Exception { Duration.millis(100), ClosingBehavior.FIRE_IF_NON_EMPTY); - tester.advanceInputWatermark(new Instant(0)); + tester.advanceInputWatermark(Instant.ofEpochMilli(0)); when(mockTriggerStateMachine.shouldFire(anyTriggerContext())).thenReturn(true); injectElement(tester, 1); assertThat( @@ -1155,7 +1162,7 @@ public void testPaneInfoAllStates() throws Exception { when(mockTriggerStateMachine.shouldFire(anyTriggerContext())).thenReturn(false); tester.setAutoAdvanceOutputWatermark(false); - tester.advanceInputWatermark(new Instant(15)); + tester.advanceInputWatermark(Instant.ofEpochMilli(15)); when(mockTriggerStateMachine.shouldFire(anyTriggerContext())).thenReturn(true); injectElement(tester, 3); @@ -1197,9 +1204,10 @@ public void testPaneInfoAllStatesAfterWatermark() throws Exception { .withTimestampCombiner(TimestampCombiner.EARLIEST) .withClosingBehavior(ClosingBehavior.FIRE_ALWAYS)); - tester.advanceInputWatermark(new Instant(0)); + tester.advanceInputWatermark(Instant.ofEpochMilli(0)); tester.injectElements( - TimestampedValue.of(1, new Instant(1)), TimestampedValue.of(2, new Instant(2))); + TimestampedValue.of(1, Instant.ofEpochMilli(1)), + TimestampedValue.of(2, Instant.ofEpochMilli(2))); List>> output = tester.extractOutput(); assertThat( @@ -1209,7 +1217,7 @@ public void testPaneInfoAllStatesAfterWatermark() throws Exception { PaneInfo.createPane(true, false, Timing.EARLY, 0, -1)))); assertThat(output, contains(isSingleWindowedValue(containsInAnyOrder(1, 2), 1, 0, 10))); - tester.advanceInputWatermark(new Instant(50)); + tester.advanceInputWatermark(Instant.ofEpochMilli(50)); // We should get the ON_TIME pane even though it is empty, // because we have an AfterWatermark.pastEndOfWindow() trigger. @@ -1222,7 +1230,7 @@ public void testPaneInfoAllStatesAfterWatermark() throws Exception { assertThat(output, contains(isSingleWindowedValue(emptyIterable(), 9, 0, 10))); // We should get the final pane even though it is empty. - tester.advanceInputWatermark(new Instant(150)); + tester.advanceInputWatermark(Instant.ofEpochMilli(150)); output = tester.extractOutput(); assertThat( output, @@ -1245,11 +1253,12 @@ public void noEmptyPanesFinalIfNonEmpty() throws Exception { .withTimestampCombiner(TimestampCombiner.EARLIEST) .withClosingBehavior(ClosingBehavior.FIRE_IF_NON_EMPTY)); - tester.advanceInputWatermark(new Instant(0)); + tester.advanceInputWatermark(Instant.ofEpochMilli(0)); tester.injectElements( - TimestampedValue.of(1, new Instant(1)), TimestampedValue.of(2, new Instant(2))); - tester.advanceInputWatermark(new Instant(20)); - tester.advanceInputWatermark(new Instant(250)); + TimestampedValue.of(1, Instant.ofEpochMilli(1)), + TimestampedValue.of(2, Instant.ofEpochMilli(2))); + tester.advanceInputWatermark(Instant.ofEpochMilli(20)); + tester.advanceInputWatermark(Instant.ofEpochMilli(250)); List>> output = tester.extractOutput(); assertThat( @@ -1275,11 +1284,12 @@ public void noEmptyPanesFinalAlways() throws Exception { .withTimestampCombiner(TimestampCombiner.EARLIEST) .withClosingBehavior(ClosingBehavior.FIRE_ALWAYS)); - tester.advanceInputWatermark(new Instant(0)); + tester.advanceInputWatermark(Instant.ofEpochMilli(0)); tester.injectElements( - TimestampedValue.of(1, new Instant(1)), TimestampedValue.of(2, new Instant(2))); - tester.advanceInputWatermark(new Instant(20)); - tester.advanceInputWatermark(new Instant(250)); + TimestampedValue.of(1, Instant.ofEpochMilli(1)), + TimestampedValue.of(2, Instant.ofEpochMilli(2))); + tester.advanceInputWatermark(Instant.ofEpochMilli(20)); + tester.advanceInputWatermark(Instant.ofEpochMilli(250)); List>> output = tester.extractOutput(); assertThat( @@ -1311,16 +1321,17 @@ public void testNoWatermarkTriggerNoHold() throws Exception { // First, an element comes in on time in [0, 10) but ReduceFnRunner should // not set a hold or timer for 9. That is the trigger's job. - IntervalWindow expectedWindow = new IntervalWindow(new Instant(0), new Instant(10)); - tester.advanceInputWatermark(new Instant(0)); - tester.advanceProcessingTime(new Instant(0)); + IntervalWindow expectedWindow = + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10)); + tester.advanceInputWatermark(Instant.ofEpochMilli(0)); + tester.advanceProcessingTime(Instant.ofEpochMilli(0)); - tester.injectElements(TimestampedValue.of(1, new Instant(1))); + tester.injectElements(TimestampedValue.of(1, Instant.ofEpochMilli(1))); // Since some data arrived, the element hold will be the end of the window. assertThat(tester.getWatermarkHold(), equalTo(expectedWindow.maxTimestamp())); - tester.advanceProcessingTime(new Instant(6000)); + tester.advanceProcessingTime(Instant.ofEpochMilli(6000)); // Sanity check; we aren't trying to verify output in this test assertThat(tester.getOutputSize(), equalTo(1)); @@ -1332,7 +1343,7 @@ public void testNoWatermarkTriggerNoHold() throws Exception { tester.advanceInputWatermark(expectedWindow.maxTimestamp().plus(Duration.standardHours(1))); // Now late data arrives - tester.injectElements(TimestampedValue.of(3, new Instant(3))); + tester.injectElements(TimestampedValue.of(3, Instant.ofEpochMilli(3))); // The ReduceFnRunner should set a GC hold since the element was too late and its timestamp // will be ignored for the purposes of the watermark hold @@ -1354,9 +1365,10 @@ public void testPaneInfoAllStatesAfterWatermarkAccumulating() throws Exception { .withTimestampCombiner(TimestampCombiner.EARLIEST) .withClosingBehavior(ClosingBehavior.FIRE_ALWAYS)); - tester.advanceInputWatermark(new Instant(0)); + tester.advanceInputWatermark(Instant.ofEpochMilli(0)); tester.injectElements( - TimestampedValue.of(1, new Instant(1)), TimestampedValue.of(2, new Instant(2))); + TimestampedValue.of(1, Instant.ofEpochMilli(1)), + TimestampedValue.of(2, Instant.ofEpochMilli(2))); List>> output = tester.extractOutput(); assertThat( @@ -1366,7 +1378,7 @@ public void testPaneInfoAllStatesAfterWatermarkAccumulating() throws Exception { PaneInfo.createPane(true, false, Timing.EARLY, 0, -1)))); assertThat(output, contains(isSingleWindowedValue(containsInAnyOrder(1, 2), 1, 0, 10))); - tester.advanceInputWatermark(new Instant(50)); + tester.advanceInputWatermark(Instant.ofEpochMilli(50)); // We should get the ON_TIME pane even though it is empty, // because we have an AfterWatermark.pastEndOfWindow() trigger. @@ -1379,7 +1391,7 @@ public void testPaneInfoAllStatesAfterWatermarkAccumulating() throws Exception { assertThat(output, contains(isSingleWindowedValue(containsInAnyOrder(1, 2), 9, 0, 10))); // We should get the final pane even though it is empty. - tester.advanceInputWatermark(new Instant(150)); + tester.advanceInputWatermark(Instant.ofEpochMilli(150)); output = tester.extractOutput(); assertThat( output, @@ -1400,11 +1412,12 @@ public void testPaneInfoFinalAndOnTime() throws Exception { .withAllowedLateness(Duration.millis(100)) .withClosingBehavior(ClosingBehavior.FIRE_ALWAYS)); - tester.advanceInputWatermark(new Instant(0)); + tester.advanceInputWatermark(Instant.ofEpochMilli(0)); // Should trigger due to element count tester.injectElements( - TimestampedValue.of(1, new Instant(1)), TimestampedValue.of(2, new Instant(2))); + TimestampedValue.of(1, Instant.ofEpochMilli(1)), + TimestampedValue.of(2, Instant.ofEpochMilli(2))); assertThat( tester.extractOutput(), @@ -1412,7 +1425,7 @@ public void testPaneInfoFinalAndOnTime() throws Exception { WindowMatchers.valueWithPaneInfo( PaneInfo.createPane(true, false, Timing.EARLY, 0, -1)))); - tester.advanceInputWatermark(new Instant(150)); + tester.advanceInputWatermark(Instant.ofEpochMilli(150)); assertThat( tester.extractOutput(), contains( @@ -1430,7 +1443,7 @@ public void testPaneInfoSkipToFinish() throws Exception { Duration.millis(100), ClosingBehavior.FIRE_IF_NON_EMPTY); - tester.advanceInputWatermark(new Instant(0)); + tester.advanceInputWatermark(Instant.ofEpochMilli(0)); when(mockTriggerStateMachine.shouldFire(anyTriggerContext())).thenReturn(true); triggerShouldFinish(mockTriggerStateMachine); injectElement(tester, 1); @@ -1449,7 +1462,7 @@ public void testPaneInfoSkipToNonSpeculativeAndFinish() throws Exception { Duration.millis(100), ClosingBehavior.FIRE_IF_NON_EMPTY); - tester.advanceInputWatermark(new Instant(15)); + tester.advanceInputWatermark(Instant.ofEpochMilli(15)); when(mockTriggerStateMachine.shouldFire(anyTriggerContext())).thenReturn(true); triggerShouldFinish(mockTriggerStateMachine); injectElement(tester, 1); @@ -1473,11 +1486,11 @@ public void testMergeBeforeFinalizing() throws Exception { // All on time data, verify watermark hold. // These two windows should pre-merge immediately to [1, 20) tester.injectElements( - TimestampedValue.of(1, new Instant(1)), // in [1, 11) - TimestampedValue.of(10, new Instant(10))); // in [10, 20) + TimestampedValue.of(1, Instant.ofEpochMilli(1)), // in [1, 11) + TimestampedValue.of(10, Instant.ofEpochMilli(10))); // in [10, 20) // And this should fire the end-of-window timer - tester.advanceInputWatermark(new Instant(100)); + tester.advanceInputWatermark(Instant.ofEpochMilli(100)); List>> output = tester.extractOutput(); assertThat(output.size(), equalTo(1)); @@ -1510,16 +1523,16 @@ public void testMergingWithCloseBeforeGC() throws Exception { // Two elements in two overlapping session windows. tester.injectElements( - TimestampedValue.of(1, new Instant(1)), // in [1, 11) - TimestampedValue.of(10, new Instant(10))); // in [10, 20) + TimestampedValue.of(1, Instant.ofEpochMilli(1)), // in [1, 11) + TimestampedValue.of(10, Instant.ofEpochMilli(10))); // in [10, 20) // Close the trigger, but the gargbage collection timer is still pending. when(mockTriggerStateMachine.shouldFire(anyTriggerContext())).thenReturn(true); triggerShouldFinish(mockTriggerStateMachine); - tester.advanceInputWatermark(new Instant(30)); + tester.advanceInputWatermark(Instant.ofEpochMilli(30)); // Now the garbage collection timer will fire, finding the trigger already closed. - tester.advanceInputWatermark(new Instant(100)); + tester.advanceInputWatermark(Instant.ofEpochMilli(100)); List>> output = tester.extractOutput(); assertThat(output.size(), equalTo(1)); @@ -1548,16 +1561,18 @@ public void testMergingWithCloseTrigger() throws Exception { ClosingBehavior.FIRE_IF_NON_EMPTY); // Create a new merged session window. - IntervalWindow mergedWindow = new IntervalWindow(new Instant(1), new Instant(12)); + IntervalWindow mergedWindow = + new IntervalWindow(Instant.ofEpochMilli(1), Instant.ofEpochMilli(12)); tester.injectElements( - TimestampedValue.of(1, new Instant(1)), TimestampedValue.of(2, new Instant(2))); + TimestampedValue.of(1, Instant.ofEpochMilli(1)), + TimestampedValue.of(2, Instant.ofEpochMilli(2))); // Force the trigger to be closed for the merged window. when(mockTriggerStateMachine.shouldFire(anyTriggerContext())).thenReturn(true); triggerShouldFinish(mockTriggerStateMachine); // Fire and end-of-window timer as though the trigger set it - tester.advanceInputWatermark(new Instant(13)); + tester.advanceInputWatermark(Instant.ofEpochMilli(13)); tester.fireTimer(mergedWindow, mergedWindow.maxTimestamp(), TimeDomain.EVENT_TIME); // Trigger is now closed. @@ -1567,7 +1582,8 @@ public void testMergingWithCloseTrigger() throws Exception { // Revisit the same session window. tester.injectElements( - TimestampedValue.of(1, new Instant(1)), TimestampedValue.of(2, new Instant(2))); + TimestampedValue.of(1, Instant.ofEpochMilli(1)), + TimestampedValue.of(2, Instant.ofEpochMilli(2))); // Trigger is still closed. assertTrue(tester.isMarkedFinished(mergedWindow)); @@ -1588,26 +1604,27 @@ public void testMergingWithReusedWindow() throws Exception { allowedLateness, ClosingBehavior.FIRE_IF_NON_EMPTY); - IntervalWindow mergedWindow = new IntervalWindow(new Instant(1), new Instant(11)); + IntervalWindow mergedWindow = + new IntervalWindow(Instant.ofEpochMilli(1), Instant.ofEpochMilli(11)); // One elements in one session window. - tester.injectElements(TimestampedValue.of(1, new Instant(1))); // in [1, 11), gc at 21. + tester.injectElements(TimestampedValue.of(1, Instant.ofEpochMilli(1))); // in [1, 11), gc at 21. // Close the trigger, but the gargbage collection timer is still pending. when(mockTriggerStateMachine.shouldFire(anyTriggerContext())).thenReturn(true); triggerShouldFinish(mockTriggerStateMachine); - tester.advanceInputWatermark(new Instant(15)); + tester.advanceInputWatermark(Instant.ofEpochMilli(15)); tester.fireTimer(mergedWindow, mergedWindow.maxTimestamp(), TimeDomain.EVENT_TIME); // Another element in the same session window. // Should be discarded with 'window closed'. - tester.injectElements(TimestampedValue.of(1, new Instant(1))); // in [1, 11), gc at 21. + tester.injectElements(TimestampedValue.of(1, Instant.ofEpochMilli(1))); // in [1, 11), gc at 21. // And nothing should be left in the active window state. assertTrue(tester.hasNoActiveWindows()); // Now the garbage collection timer will fire, finding the trigger already closed. - tester.advanceInputWatermark(new Instant(100)); + tester.advanceInputWatermark(Instant.ofEpochMilli(100)); List>> output = tester.extractOutput(); assertThat(output.size(), equalTo(1)); @@ -1615,7 +1632,7 @@ public void testMergingWithReusedWindow() throws Exception { output.get(0), isSingleWindowedValue( containsInAnyOrder(1), - equalTo(new Instant(1)), // timestamp + equalTo(Instant.ofEpochMilli(1)), // timestamp equalTo((BoundedWindow) mergedWindow))); assertThat( @@ -1642,19 +1659,19 @@ public void testMergingWithClosedRepresentative() throws Exception { when(mockTriggerStateMachine.shouldFire(anyTriggerContext())).thenReturn(true); triggerShouldFinish(mockTriggerStateMachine); tester.injectElements( - TimestampedValue.of(1, new Instant(1)), // in [1, 11), gc at 21. - TimestampedValue.of(8, new Instant(8))); // in [8, 18), gc at 28. + TimestampedValue.of(1, Instant.ofEpochMilli(1)), // in [1, 11), gc at 21. + TimestampedValue.of(8, Instant.ofEpochMilli(8))); // in [8, 18), gc at 28. // More elements into the same merged session window. // It has not yet been gced. // Should be discarded with 'window closed'. tester.injectElements( - TimestampedValue.of(1, new Instant(1)), // in [1, 11), gc at 21. - TimestampedValue.of(2, new Instant(2)), // in [2, 12), gc at 22. - TimestampedValue.of(8, new Instant(8))); // in [8, 18), gc at 28. + TimestampedValue.of(1, Instant.ofEpochMilli(1)), // in [1, 11), gc at 21. + TimestampedValue.of(2, Instant.ofEpochMilli(2)), // in [2, 12), gc at 22. + TimestampedValue.of(8, Instant.ofEpochMilli(8))); // in [8, 18), gc at 28. // Now the garbage collection timer will fire, finding the trigger already closed. - tester.advanceInputWatermark(new Instant(100)); + tester.advanceInputWatermark(Instant.ofEpochMilli(100)); List>> output = tester.extractOutput(); @@ -1687,16 +1704,16 @@ public void testMergingWithClosedDoesNotPoison() throws Exception { // 1 element, force its trigger to close. when(mockTriggerStateMachine.shouldFire(anyTriggerContext())).thenReturn(true); triggerShouldFinish(mockTriggerStateMachine); - tester.injectElements(TimestampedValue.of(2, new Instant(2))); + tester.injectElements(TimestampedValue.of(2, Instant.ofEpochMilli(2))); // 3 elements, one already closed. when(mockTriggerStateMachine.shouldFire(anyTriggerContext())).thenReturn(false); tester.injectElements( - TimestampedValue.of(1, new Instant(1)), - TimestampedValue.of(2, new Instant(2)), - TimestampedValue.of(3, new Instant(3))); + TimestampedValue.of(1, Instant.ofEpochMilli(1)), + TimestampedValue.of(2, Instant.ofEpochMilli(2)), + TimestampedValue.of(3, Instant.ofEpochMilli(3))); - tester.advanceInputWatermark(new Instant(100)); + tester.advanceInputWatermark(Instant.ofEpochMilli(100)); List>> output = tester.extractOutput(); assertThat(output.size(), equalTo(2)); @@ -1739,9 +1756,9 @@ public void testDropDataMultipleWindowsFinishedTrigger() throws Exception { tester.injectElements( // assigned to [-60, 40), [-30, 70), [0, 100) - TimestampedValue.of(10, new Instant(23)), + TimestampedValue.of(10, Instant.ofEpochMilli(23)), // assigned to [-30, 70), [0, 100), [30, 130) - TimestampedValue.of(12, new Instant(40))); + TimestampedValue.of(12, Instant.ofEpochMilli(40))); long droppedElements = container @@ -1750,11 +1767,11 @@ public void testDropDataMultipleWindowsFinishedTrigger() throws Exception { .getCumulative(); assertEquals(0, droppedElements); - tester.advanceInputWatermark(new Instant(70)); + tester.advanceInputWatermark(Instant.ofEpochMilli(70)); tester.injectElements( // assigned to [-30, 70), [0, 100), [30, 130) // but [-30, 70) is closed by the trigger - TimestampedValue.of(14, new Instant(60))); + TimestampedValue.of(14, Instant.ofEpochMilli(60))); droppedElements = container @@ -1763,10 +1780,10 @@ public void testDropDataMultipleWindowsFinishedTrigger() throws Exception { .getCumulative(); assertEquals(1, droppedElements); - tester.advanceInputWatermark(new Instant(130)); + tester.advanceInputWatermark(Instant.ofEpochMilli(130)); // assigned to [-30, 70), [0, 100), [30, 130) // but they are all closed - tester.injectElements(TimestampedValue.of(16, new Instant(40))); + tester.injectElements(TimestampedValue.of(16, Instant.ofEpochMilli(40))); droppedElements = container @@ -1793,15 +1810,15 @@ public void testIdempotentEmptyPanesDiscarding() throws Exception { // Inject a couple of on-time elements and fire at the window end. injectElement(tester, 1); injectElement(tester, 2); - tester.advanceInputWatermark(new Instant(12)); + tester.advanceInputWatermark(Instant.ofEpochMilli(12)); // Fire the on-time paneInfo when(mockTriggerStateMachine.shouldFire(anyTriggerContext())).thenReturn(true); - tester.fireTimer(firstWindow, new Instant(9), TimeDomain.EVENT_TIME); + tester.fireTimer(firstWindow, Instant.ofEpochMilli(9), TimeDomain.EVENT_TIME); // Fire another timer (with no data, so it's an uninteresting pane that should not be output). when(mockTriggerStateMachine.shouldFire(anyTriggerContext())).thenReturn(true); - tester.fireTimer(firstWindow, new Instant(9), TimeDomain.EVENT_TIME); + tester.fireTimer(firstWindow, Instant.ofEpochMilli(9), TimeDomain.EVENT_TIME); // Finish it off with another datum. when(mockTriggerStateMachine.shouldFire(anyTriggerContext())).thenReturn(true); @@ -1848,11 +1865,11 @@ public void testIdempotentEmptyPanesAccumulating() throws Exception { // Inject a couple of on-time elements and fire at the window end. injectElement(tester, 1); injectElement(tester, 2); - tester.advanceInputWatermark(new Instant(12)); + tester.advanceInputWatermark(Instant.ofEpochMilli(12)); // Trigger the on-time paneInfo when(mockTriggerStateMachine.shouldFire(anyTriggerContext())).thenReturn(true); - tester.fireTimer(firstWindow, new Instant(9), TimeDomain.EVENT_TIME); + tester.fireTimer(firstWindow, Instant.ofEpochMilli(9), TimeDomain.EVENT_TIME); List>> output = tester.extractOutput(); assertThat(output.size(), equalTo(1)); assertThat(output.get(0), isSingleWindowedValue(containsInAnyOrder(1, 2), 1, 0, 10)); @@ -1863,7 +1880,7 @@ public void testIdempotentEmptyPanesAccumulating() throws Exception { // Fire another timer with no data; the empty pane should not be output even though the // trigger is ready to fire when(mockTriggerStateMachine.shouldFire(anyTriggerContext())).thenReturn(true); - tester.fireTimer(firstWindow, new Instant(9), TimeDomain.EVENT_TIME); + tester.fireTimer(firstWindow, Instant.ofEpochMilli(9), TimeDomain.EVENT_TIME); assertThat(tester.extractOutput().size(), equalTo(0)); // Finish it off with another datum, which is late @@ -1914,21 +1931,21 @@ public void testEmptyOnTimeFromOrFinally() throws Exception { ReduceFnTester tester = ReduceFnTester.combining(strategy, Sum.ofIntegers(), VarIntCoder.of()); - tester.advanceInputWatermark(new Instant(0)); - tester.advanceProcessingTime(new Instant(0)); + tester.advanceInputWatermark(Instant.ofEpochMilli(0)); + tester.advanceProcessingTime(Instant.ofEpochMilli(0)); // Processing time timer for 5 tester.injectElements( - TimestampedValue.of(1, new Instant(1)), - TimestampedValue.of(1, new Instant(3)), - TimestampedValue.of(1, new Instant(7)), - TimestampedValue.of(1, new Instant(5))); + TimestampedValue.of(1, Instant.ofEpochMilli(1)), + TimestampedValue.of(1, Instant.ofEpochMilli(3)), + TimestampedValue.of(1, Instant.ofEpochMilli(7)), + TimestampedValue.of(1, Instant.ofEpochMilli(5))); // Should fire early paneInfo - tester.advanceProcessingTime(new Instant(6)); + tester.advanceProcessingTime(Instant.ofEpochMilli(6)); // Should fire empty on time paneInfo - tester.advanceInputWatermark(new Instant(11)); + tester.advanceInputWatermark(Instant.ofEpochMilli(11)); List> output = tester.extractOutput(); assertEquals(2, output.size()); @@ -1966,24 +1983,24 @@ public void testEmptyOnTimeWithOnTimeBehaviorFireIfNonEmpty() throws Exception { ReduceFnTester tester = ReduceFnTester.combining(strategy, Sum.ofIntegers(), VarIntCoder.of()); - tester.advanceInputWatermark(new Instant(0)); - tester.advanceProcessingTime(new Instant(0)); + tester.advanceInputWatermark(Instant.ofEpochMilli(0)); + tester.advanceProcessingTime(Instant.ofEpochMilli(0)); // Processing time timer for 5 tester.injectElements( - TimestampedValue.of(1, new Instant(1)), - TimestampedValue.of(1, new Instant(3)), - TimestampedValue.of(1, new Instant(7)), - TimestampedValue.of(1, new Instant(5))); + TimestampedValue.of(1, Instant.ofEpochMilli(1)), + TimestampedValue.of(1, Instant.ofEpochMilli(3)), + TimestampedValue.of(1, Instant.ofEpochMilli(7)), + TimestampedValue.of(1, Instant.ofEpochMilli(5))); // Should fire early paneInfo - tester.advanceProcessingTime(new Instant(6)); + tester.advanceProcessingTime(Instant.ofEpochMilli(6)); // Should not fire empty on time paneInfo - tester.advanceInputWatermark(new Instant(11)); + tester.advanceInputWatermark(Instant.ofEpochMilli(11)); // Should fire final GC paneInfo - tester.advanceInputWatermark(new Instant(10 + 100)); + tester.advanceInputWatermark(Instant.ofEpochMilli(10 + 100)); List> output = tester.extractOutput(); assertEquals(2, output.size()); @@ -2018,13 +2035,13 @@ public void testEmptyOnTimeWithOnTimeBehaviorBackwardCompatibility() throws Exce ReduceFnTester tester = ReduceFnTester.combining(strategy, Sum.ofIntegers(), VarIntCoder.of()); - tester.advanceInputWatermark(new Instant(0)); - tester.advanceProcessingTime(new Instant(0)); + tester.advanceInputWatermark(Instant.ofEpochMilli(0)); + tester.advanceProcessingTime(Instant.ofEpochMilli(0)); - tester.injectElements(TimestampedValue.of(1, new Instant(1))); + tester.injectElements(TimestampedValue.of(1, Instant.ofEpochMilli(1))); // Should fire empty on time isFinished paneInfo - tester.advanceInputWatermark(new Instant(11)); + tester.advanceInputWatermark(Instant.ofEpochMilli(11)); List> output = tester.extractOutput(); assertEquals(2, output.size()); @@ -2062,25 +2079,25 @@ public void testEmptyOnTimeWithOnTimeBehaviorFireIfNonEmptyAndLateData() throws ReduceFnTester tester = ReduceFnTester.combining(strategy, Sum.ofIntegers(), VarIntCoder.of()); - tester.advanceInputWatermark(new Instant(0)); - tester.advanceProcessingTime(new Instant(0)); + tester.advanceInputWatermark(Instant.ofEpochMilli(0)); + tester.advanceProcessingTime(Instant.ofEpochMilli(0)); // Processing time timer for 5 tester.injectElements( - TimestampedValue.of(1, new Instant(1)), - TimestampedValue.of(1, new Instant(3)), - TimestampedValue.of(1, new Instant(7)), - TimestampedValue.of(1, new Instant(5))); + TimestampedValue.of(1, Instant.ofEpochMilli(1)), + TimestampedValue.of(1, Instant.ofEpochMilli(3)), + TimestampedValue.of(1, Instant.ofEpochMilli(7)), + TimestampedValue.of(1, Instant.ofEpochMilli(5))); // Should fire early paneInfo - tester.advanceProcessingTime(new Instant(6)); + tester.advanceProcessingTime(Instant.ofEpochMilli(6)); // Should not fire empty on time paneInfo - tester.advanceInputWatermark(new Instant(11)); + tester.advanceInputWatermark(Instant.ofEpochMilli(11)); // Processing late data, and should fire late paneInfo - tester.injectElements(TimestampedValue.of(1, new Instant(9))); - tester.advanceProcessingTime(new Instant(6 + 25 + 1)); + tester.injectElements(TimestampedValue.of(1, Instant.ofEpochMilli(9))); + tester.advanceProcessingTime(Instant.ofEpochMilli(6 + 25 + 1)); List> output = tester.extractOutput(); assertEquals(2, output.size()); @@ -2121,41 +2138,42 @@ public void testProcessingTime() throws Exception { ReduceFnTester tester = ReduceFnTester.combining(strategy, Sum.ofIntegers(), VarIntCoder.of()); - tester.advanceInputWatermark(new Instant(0)); - tester.advanceProcessingTime(new Instant(0)); + tester.advanceInputWatermark(Instant.ofEpochMilli(0)); + tester.advanceProcessingTime(Instant.ofEpochMilli(0)); tester.injectElements( - TimestampedValue.of(1, new Instant(1)), - TimestampedValue.of(1, new Instant(3)), - TimestampedValue.of(1, new Instant(7)), - TimestampedValue.of(1, new Instant(5))); + TimestampedValue.of(1, Instant.ofEpochMilli(1)), + TimestampedValue.of(1, Instant.ofEpochMilli(3)), + TimestampedValue.of(1, Instant.ofEpochMilli(7)), + TimestampedValue.of(1, Instant.ofEpochMilli(5))); // 4 elements all at processing time 0 - tester.advanceProcessingTime(new Instant(6)); // fire [1,3,7,5] since 6 > 0 + 5 + tester.advanceProcessingTime(Instant.ofEpochMilli(6)); // fire [1,3,7,5] since 6 > 0 + 5 tester.injectElements( - TimestampedValue.of(1, new Instant(8)), TimestampedValue.of(1, new Instant(4))); + TimestampedValue.of(1, Instant.ofEpochMilli(8)), + TimestampedValue.of(1, Instant.ofEpochMilli(4))); // 6 elements - tester.advanceInputWatermark(new Instant(11)); // fire [1,3,7,5,8,4] since 11 > 9 + tester.advanceInputWatermark(Instant.ofEpochMilli(11)); // fire [1,3,7,5,8,4] since 11 > 9 tester.injectElements( - TimestampedValue.of(1, new Instant(8)), - TimestampedValue.of(1, new Instant(4)), - TimestampedValue.of(1, new Instant(5))); + TimestampedValue.of(1, Instant.ofEpochMilli(8)), + TimestampedValue.of(1, Instant.ofEpochMilli(4)), + TimestampedValue.of(1, Instant.ofEpochMilli(5))); // 9 elements - tester.advanceInputWatermark(new Instant(12)); - tester.injectElements(TimestampedValue.of(1, new Instant(3))); + tester.advanceInputWatermark(Instant.ofEpochMilli(12)); + tester.injectElements(TimestampedValue.of(1, Instant.ofEpochMilli(3))); // 10 elements - tester.advanceProcessingTime(new Instant(15)); - tester.injectElements(TimestampedValue.of(1, new Instant(5))); + tester.advanceProcessingTime(Instant.ofEpochMilli(15)); + tester.injectElements(TimestampedValue.of(1, Instant.ofEpochMilli(5))); // 11 elements - tester.advanceProcessingTime(new Instant(32)); // fire since 32 > 6 + 25 + tester.advanceProcessingTime(Instant.ofEpochMilli(32)); // fire since 32 > 6 + 25 - tester.injectElements(TimestampedValue.of(1, new Instant(3))); + tester.injectElements(TimestampedValue.of(1, Instant.ofEpochMilli(3))); // 12 elements // fire [1,3,7,5,8,4,8,4,5,3,5,3] since 125 > 6 + 25 - tester.advanceInputWatermark(new Instant(125)); + tester.advanceInputWatermark(Instant.ofEpochMilli(125)); List> output = tester.extractOutput(); assertEquals(4, output.size()); @@ -2191,11 +2209,11 @@ public void fireNonEmptyOnDrainInGlobalWindow() throws Exception { .withTrigger(Repeatedly.forever(AfterPane.elementCountAtLeast(3))) .withMode(AccumulationMode.DISCARDING_FIRED_PANES)); - tester.advanceInputWatermark(new Instant(0)); + tester.advanceInputWatermark(Instant.ofEpochMilli(0)); final int n = 20; for (int i = 0; i < n; i++) { - tester.injectElements(TimestampedValue.of(i, new Instant(i))); + tester.injectElements(TimestampedValue.of(i, Instant.ofEpochMilli(i))); } List>> output = tester.extractOutput(); @@ -2232,10 +2250,10 @@ public void fireEmptyOnDrainInGlobalWindowIfRequested() throws Exception { final int n = 20; for (int i = 0; i < n; i++) { - tester.advanceProcessingTime(new Instant(i)); - tester.injectElements(TimestampedValue.of(i, new Instant(i))); + tester.advanceProcessingTime(Instant.ofEpochMilli(i)); + tester.injectElements(TimestampedValue.of(i, Instant.ofEpochMilli(i))); } - tester.advanceProcessingTime(new Instant(n + 4)); + tester.advanceProcessingTime(Instant.ofEpochMilli(n + 4)); List>> output = tester.extractOutput(); assertEquals((n + 3) / 4, output.size()); for (int i = 0; i < output.size(); i++) { @@ -2269,21 +2287,21 @@ public void setGarbageCollectionHoldOnLateElements() throws Exception { .withAllowedLateness(Duration.millis(100)) .withClosingBehavior(ClosingBehavior.FIRE_IF_NON_EMPTY)); - tester.advanceInputWatermark(new Instant(0)); - tester.advanceOutputWatermark(new Instant(0)); - tester.injectElements(TimestampedValue.of(1, new Instant(1))); + tester.advanceInputWatermark(Instant.ofEpochMilli(0)); + tester.advanceOutputWatermark(Instant.ofEpochMilli(0)); + tester.injectElements(TimestampedValue.of(1, Instant.ofEpochMilli(1))); // Fire ON_TIME pane @ 9 with 1 - tester.advanceInputWatermark(new Instant(109)); - tester.advanceOutputWatermark(new Instant(109)); - tester.injectElements(TimestampedValue.of(2, new Instant(2))); + tester.advanceInputWatermark(Instant.ofEpochMilli(109)); + tester.advanceOutputWatermark(Instant.ofEpochMilli(109)); + tester.injectElements(TimestampedValue.of(2, Instant.ofEpochMilli(2))); // We should have set a garbage collection hold for the final pane. Instant hold = tester.getWatermarkHold(); - assertEquals(new Instant(109), hold); + assertEquals(Instant.ofEpochMilli(109), hold); - tester.advanceInputWatermark(new Instant(110)); - tester.advanceOutputWatermark(new Instant(110)); + tester.advanceInputWatermark(Instant.ofEpochMilli(110)); + tester.advanceOutputWatermark(Instant.ofEpochMilli(110)); // Fire final LATE pane @ 9 with 2 diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SideInputHandlerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SideInputHandlerTest.java index 293a7fe72c7e..002090c4bba8 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SideInputHandlerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SideInputHandlerTest.java @@ -95,9 +95,11 @@ public void testIsReady() { ImmutableList.of(view1, view2), InMemoryStateInternals.forKey(null)); // Adjacent fixed windows - IntervalWindow firstWindow = new IntervalWindow(new Instant(0), new Instant(view1WindowSize)); + IntervalWindow firstWindow = + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(view1WindowSize)); IntervalWindow secondWindow = - new IntervalWindow(new Instant(view1WindowSize), new Instant(view1WindowSize * 2)); + new IntervalWindow( + Instant.ofEpochMilli(view1WindowSize), Instant.ofEpochMilli(view1WindowSize * 2)); // side input should not yet be ready in first window assertFalse(sideInputHandler.isReady(view1, firstWindow)); @@ -107,7 +109,7 @@ public void testIsReady() { view1, valuesInWindow( materializeValuesFor(view1.getPipeline().getOptions(), View.asIterable(), "Hello"), - new Instant(0), + Instant.ofEpochMilli(0), firstWindow)); // now side input should be ready in first window @@ -133,14 +135,15 @@ public void testNewInputReplacesPreviousInput() { SideInputHandler sideInputHandler = new SideInputHandler(ImmutableList.of(view), InMemoryStateInternals.forKey(null)); - IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(view1WindowSize)); + IntervalWindow window = + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(view1WindowSize)); // add a first value for view sideInputHandler.addSideInputValue( view, valuesInWindow( materializeValuesFor(view.getPipeline().getOptions(), View.asIterable(), "Hello"), - new Instant(0), + Instant.ofEpochMilli(0), window)); assertThat(sideInputHandler.get(view, window), contains("Hello")); @@ -151,7 +154,7 @@ public void testNewInputReplacesPreviousInput() { valuesInWindow( materializeValuesFor( view.getPipeline().getOptions(), View.asIterable(), "Ciao", "Buongiorno"), - new Instant(0), + Instant.ofEpochMilli(0), window)); assertThat(sideInputHandler.get(view, window), contains("Ciao", "Buongiorno")); @@ -170,16 +173,18 @@ public void testMultipleWindows() { new SideInputHandler(ImmutableList.of(view1), InMemoryStateInternals.forKey(null)); // two windows that we'll later use for adding elements/retrieving side input - IntervalWindow firstWindow = new IntervalWindow(new Instant(0), new Instant(view1WindowSize)); + IntervalWindow firstWindow = + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(view1WindowSize)); IntervalWindow secondWindow = - new IntervalWindow(new Instant(view1WindowSize), new Instant(view1WindowSize * 2)); + new IntervalWindow( + Instant.ofEpochMilli(view1WindowSize), Instant.ofEpochMilli(view1WindowSize * 2)); // add a first value for view1 in the first window sideInputHandler.addSideInputValue( view1, valuesInWindow( materializeValuesFor(view1.getPipeline().getOptions(), View.asIterable(), "Hello"), - new Instant(0), + Instant.ofEpochMilli(0), firstWindow)); assertThat(sideInputHandler.get(view1, firstWindow), contains("Hello")); @@ -190,7 +195,7 @@ public void testMultipleWindows() { valuesInWindow( materializeValuesFor( view1.getPipeline().getOptions(), View.asIterable(), "Arrivederci"), - new Instant(0), + Instant.ofEpochMilli(0), secondWindow)); assertThat(sideInputHandler.get(view1, secondWindow), contains("Arrivederci")); @@ -217,14 +222,15 @@ public void testMultipleSideInputs() { ImmutableList.of(view1, view2), InMemoryStateInternals.forKey(null)); // two windows that we'll later use for adding elements/retrieving side input - IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(windowSize)); + IntervalWindow window = + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(windowSize)); // add value for view1 in the first window sideInputHandler.addSideInputValue( view1, valuesInWindow( materializeValuesFor(view1.getPipeline().getOptions(), View.asIterable(), "Hello"), - new Instant(0), + Instant.ofEpochMilli(0), window)); assertThat(sideInputHandler.get(view1, window), contains("Hello")); @@ -237,7 +243,7 @@ public void testMultipleSideInputs() { view2, valuesInWindow( materializeValuesFor(view2.getPipeline().getOptions(), View.asIterable(), "Salut"), - new Instant(0), + Instant.ofEpochMilli(0), window)); assertTrue(sideInputHandler.isReady(view2, window)); diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java index 49c9277314bc..029426ea271d 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java @@ -139,8 +139,8 @@ public void testOnTimerExceptionsWrappedAsUserCodeException() { "", null, GlobalWindow.INSTANCE, - new Instant(0), - new Instant(0), + Instant.ofEpochMilli(0), + Instant.ofEpochMilli(0), TimeDomain.EVENT_TIME, CausedByDrain.NORMAL); } @@ -170,7 +170,7 @@ public void testTimerSet() { Collections.emptyMap()); // Setting the timer needs the current time, as it is set relative - Instant currentTime = new Instant(42); + Instant currentTime = Instant.ofEpochMilli(42); when(mockTimerInternals.currentInputWatermarkTime()).thenReturn(currentTime); runner.processElement(WindowedValues.valueInGlobalWindow("anyValue")); @@ -255,7 +255,7 @@ public void testOnTimerCalled() { DoFnSchemaInformation.create(), Collections.emptyMap()); - Instant currentTime = new Instant(42); + Instant currentTime = Instant.ofEpochMilli(42); Duration offset = Duration.millis(37); // Mocking is not easily compatible with annotation analysis, so we manually record @@ -308,7 +308,7 @@ public void testBackwardsInTimeNoSkew() { runner.startBundle(); // An element output at the current timestamp is fine. runner.processElement( - WindowedValues.timestampedValueInGlobalWindow(Duration.ZERO, new Instant(0))); + WindowedValues.timestampedValueInGlobalWindow(Duration.ZERO, Instant.ofEpochMilli(0))); Exception exception = assertThrows( UserCodeException.class, @@ -316,7 +316,7 @@ public void testBackwardsInTimeNoSkew() { // An element output before (current time - skew) is forbidden runner.processElement( WindowedValues.timestampedValueInGlobalWindow( - Duration.millis(1L), new Instant(0))); + Duration.millis(1L), Instant.ofEpochMilli(0))); }); assertThat(exception.getCause(), isA(IllegalArgumentException.class)); @@ -326,7 +326,8 @@ public void testBackwardsInTimeNoSkew() { containsString("must be no earlier"), containsString( String.format( - "timestamp of the current input or timer (%s)", new Instant(0).toString())), + "timestamp of the current input or timer (%s)", + Instant.ofEpochMilli(0).toString())), containsString( String.format( "the allowed skew (%s)", @@ -360,7 +361,7 @@ public void testSkew() { // Outputting between "now" and "now - allowed skew" succeeds. runner.processElement( WindowedValues.timestampedValueInGlobalWindow( - Duration.standardMinutes(5L), new Instant(0))); + Duration.standardMinutes(5L), Instant.ofEpochMilli(0))); Exception exception = assertThrows( @@ -369,7 +370,7 @@ public void testSkew() { // Outputting before "now - allowed skew" fails. runner.processElement( WindowedValues.timestampedValueInGlobalWindow( - Duration.standardHours(1L), new Instant(0))); + Duration.standardHours(1L), Instant.ofEpochMilli(0))); }); assertThat(exception.getCause(), isA(IllegalArgumentException.class)); @@ -379,7 +380,8 @@ public void testSkew() { containsString("must be no earlier"), containsString( String.format( - "timestamp of the current input or timer (%s)", new Instant(0).toString())), + "timestamp of the current input or timer (%s)", + Instant.ofEpochMilli(0).toString())), containsString( String.format( "the allowed skew (%s)", @@ -411,7 +413,8 @@ public void testInfiniteSkew() { runner.startBundle(); runner.processElement( - WindowedValues.timestampedValueInGlobalWindow(Duration.millis(1L), new Instant(0))); + WindowedValues.timestampedValueInGlobalWindow( + Duration.millis(1L), Instant.ofEpochMilli(0))); runner.processElement( WindowedValues.timestampedValueInGlobalWindow( Duration.millis(1L), BoundedWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(1)))); @@ -450,7 +453,8 @@ public void testTimerBackwardsInTimeNoSkew() { runner.startBundle(); // A timer with output timestamp at the current timestamp is fine. runner.processElement( - WindowedValues.timestampedValueInGlobalWindow(KV.of("1", Duration.ZERO), new Instant(0))); + WindowedValues.timestampedValueInGlobalWindow( + KV.of("1", Duration.ZERO), Instant.ofEpochMilli(0))); Exception exception = assertThrows( @@ -459,7 +463,7 @@ public void testTimerBackwardsInTimeNoSkew() { // A timer with output timestamp before (current time - skew) is forbidden runner.processElement( WindowedValues.timestampedValueInGlobalWindow( - KV.of("2", Duration.millis(1L)), new Instant(0))); + KV.of("2", Duration.millis(1L)), Instant.ofEpochMilli(0))); }); assertThat(exception.getCause(), isA(IllegalArgumentException.class)); @@ -468,7 +472,8 @@ public void testTimerBackwardsInTimeNoSkew() { allOf( containsString("Cannot output timer with"), containsString( - String.format("output timestamp %s", new Instant(0).minus(Duration.millis(1L)))), + String.format( + "output timestamp %s", Instant.ofEpochMilli(0).minus(Duration.millis(1L)))), containsString( String.format( "allowed skew (%s)", @@ -502,7 +507,7 @@ public void testTimerSkew() { // Timer with output timestamp between "now" and "now - allowed skew" succeeds. runner.processElement( WindowedValues.timestampedValueInGlobalWindow( - KV.of("1", Duration.standardMinutes(5L)), new Instant(0))); + KV.of("1", Duration.standardMinutes(5L)), Instant.ofEpochMilli(0))); Exception exception = assertThrows( @@ -511,7 +516,7 @@ public void testTimerSkew() { // A timer with output timestamp before (current time - skew) is forbidden runner.processElement( WindowedValues.timestampedValueInGlobalWindow( - KV.of("2", Duration.standardHours(1L)), new Instant(0))); + KV.of("2", Duration.standardHours(1L)), Instant.ofEpochMilli(0))); }); assertThat(exception.getCause(), isA(IllegalArgumentException.class)); @@ -521,7 +526,8 @@ public void testTimerSkew() { containsString("Cannot output timer with"), containsString( String.format( - "output timestamp %s", new Instant(0).minus(Duration.standardHours(1L)))), + "output timestamp %s", + Instant.ofEpochMilli(0).minus(Duration.standardHours(1L)))), containsString( String.format( "allowed skew (%s)", @@ -554,7 +560,7 @@ public void testTimerInfiniteSkew() { runner.startBundle(); runner.processElement( WindowedValues.timestampedValueInGlobalWindow( - KV.of("1", Duration.millis(1L)), new Instant(0))); + KV.of("1", Duration.millis(1L)), Instant.ofEpochMilli(0))); runner.processElement( WindowedValues.timestampedValueInGlobalWindow( KV.of("2", Duration.millis(1L)), @@ -594,8 +600,8 @@ public void testOnTimerAllowedSkew() { "", null, GlobalWindow.INSTANCE, - new Instant(0), - new Instant(0), + Instant.ofEpochMilli(0), + Instant.ofEpochMilli(0), TimeDomain.EVENT_TIME, CausedByDrain.NORMAL); } @@ -627,8 +633,8 @@ public void testOnTimerNoSkew() { "", null, GlobalWindow.INSTANCE, - new Instant(0), - new Instant(0), + Instant.ofEpochMilli(0), + Instant.ofEpochMilli(0), TimeDomain.EVENT_TIME, CausedByDrain.NORMAL); }); @@ -640,7 +646,8 @@ public void testOnTimerNoSkew() { containsString("must be no earlier"), containsString( String.format( - "timestamp of the current input or timer (%s)", new Instant(0).toString())), + "timestamp of the current input or timer (%s)", + Instant.ofEpochMilli(0).toString())), containsString( String.format( "the allowed skew (%s)", @@ -753,7 +760,7 @@ private TimerSkewDoFn(Duration allowedSkew) { public void processElement(ProcessContext context, @TimerId(TIMER_ID) Timer timer) { timer .withOutputTimestamp(context.timestamp().minus(context.element().getValue())) - .set(new Instant(0)); + .set(Instant.ofEpochMilli(0)); } @OnTimer(TIMER_ID) diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunnerTest.java index 9703c3c80bd1..3dbdd752a79b 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunnerTest.java @@ -93,10 +93,10 @@ public class SimplePushbackSideInputDoFnRunnerTest { private static final long ALLOWED_LATENESS = 1; private static final IntervalWindow WINDOW_1 = - new IntervalWindow(new Instant(0), new Instant(10)); + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10)); private static final IntervalWindow WINDOW_2 = - new IntervalWindow(new Instant(10), new Instant(20)); + new IntervalWindow(Instant.ofEpochMilli(10), Instant.ofEpochMilli(20)); private static final WindowingStrategy WINDOWING_STRATEGY = WindowingStrategy.of(FixedWindows.of(Duration.millis(WINDOW_SIZE))) @@ -185,8 +185,8 @@ public void processElementSideInputNotReady() { WindowedValue oneWindow = WindowedValues.of( 2, - new Instant(-2), - new IntervalWindow(new Instant(-500L), new Instant(0L)), + Instant.ofEpochMilli(-2), + new IntervalWindow(Instant.ofEpochMilli(-500L), Instant.ofEpochMilli(0L)), PaneInfo.ON_TIME_AND_ONLY_FIRING); Iterable> oneWindowPushback = runner.processElementInReadyWindows(oneWindow); @@ -205,10 +205,10 @@ public void processElementSideInputNotReadyMultipleWindows() { WindowedValue multiWindow = WindowedValues.of( 2, - new Instant(-2), + Instant.ofEpochMilli(-2), ImmutableList.of( - new IntervalWindow(new Instant(-500L), new Instant(0L)), - new IntervalWindow(BoundedWindow.TIMESTAMP_MIN_VALUE, new Instant(250L)), + new IntervalWindow(Instant.ofEpochMilli(-500L), Instant.ofEpochMilli(0L)), + new IntervalWindow(BoundedWindow.TIMESTAMP_MIN_VALUE, Instant.ofEpochMilli(250L)), GlobalWindow.INSTANCE), PaneInfo.ON_TIME_AND_ONLY_FIRING); Iterable> multiWindowPushback = @@ -229,27 +229,29 @@ public void processElementSideInputNotReadySomeWindows() { SimplePushbackSideInputDoFnRunner runner = createRunner(ImmutableList.of(singletonView)); - IntervalWindow littleWindow = new IntervalWindow(new Instant(-500L), new Instant(0L)); + IntervalWindow littleWindow = + new IntervalWindow(Instant.ofEpochMilli(-500L), Instant.ofEpochMilli(0L)); IntervalWindow bigWindow = - new IntervalWindow(BoundedWindow.TIMESTAMP_MIN_VALUE, new Instant(250L)); + new IntervalWindow(BoundedWindow.TIMESTAMP_MIN_VALUE, Instant.ofEpochMilli(250L)); WindowedValue multiWindow = WindowedValues.of( 2, - new Instant(-2), + Instant.ofEpochMilli(-2), ImmutableList.of(littleWindow, bigWindow, GlobalWindow.INSTANCE), PaneInfo.NO_FIRING); Iterable> multiWindowPushback = runner.processElementInReadyWindows(multiWindow); assertThat( multiWindowPushback, - containsInAnyOrder(WindowedValues.timestampedValueInGlobalWindow(2, new Instant(-2L)))); + containsInAnyOrder( + WindowedValues.timestampedValueInGlobalWindow(2, Instant.ofEpochMilli(-2L)))); assertThat( underlying.inputElems, containsInAnyOrder( WindowedValues.of( - 2, new Instant(-2), ImmutableList.of(littleWindow), PaneInfo.NO_FIRING), + 2, Instant.ofEpochMilli(-2), ImmutableList.of(littleWindow), PaneInfo.NO_FIRING), WindowedValues.of( - 2, new Instant(-2), ImmutableList.of(bigWindow), PaneInfo.NO_FIRING))); + 2, Instant.ofEpochMilli(-2), ImmutableList.of(bigWindow), PaneInfo.NO_FIRING))); } @Test @@ -263,10 +265,10 @@ public void processElementSideInputReadyAllWindows() { WindowedValue multiWindow = WindowedValues.of( 2, - new Instant(-2), + Instant.ofEpochMilli(-2), ImmutableList.of( - new IntervalWindow(new Instant(-500L), new Instant(0L)), - new IntervalWindow(BoundedWindow.TIMESTAMP_MIN_VALUE, new Instant(250L)), + new IntervalWindow(Instant.ofEpochMilli(-500L), Instant.ofEpochMilli(0L)), + new IntervalWindow(BoundedWindow.TIMESTAMP_MIN_VALUE, Instant.ofEpochMilli(250L)), GlobalWindow.INSTANCE), PaneInfo.ON_TIME_AND_ONLY_FIRING); Iterable> multiWindowPushback = @@ -284,10 +286,10 @@ public void processElementNoSideInputs() { WindowedValue multiWindow = WindowedValues.of( 2, - new Instant(-2), + Instant.ofEpochMilli(-2), ImmutableList.of( - new IntervalWindow(new Instant(-500L), new Instant(0L)), - new IntervalWindow(BoundedWindow.TIMESTAMP_MIN_VALUE, new Instant(250L)), + new IntervalWindow(Instant.ofEpochMilli(-500L), Instant.ofEpochMilli(0L)), + new IntervalWindow(BoundedWindow.TIMESTAMP_MIN_VALUE, Instant.ofEpochMilli(250L)), GlobalWindow.INSTANCE), PaneInfo.ON_TIME_AND_ONLY_FIRING); Iterable> multiWindowPushback = @@ -303,8 +305,8 @@ public void testOnTimerCalled() { PushbackSideInputDoFnRunner runner = createRunner(ImmutableList.of()); String timerId = "fooTimer"; - IntervalWindow window = new IntervalWindow(new Instant(4), new Instant(16)); - Instant timestamp = new Instant(72); + IntervalWindow window = new IntervalWindow(Instant.ofEpochMilli(4), Instant.ofEpochMilli(16)); + Instant timestamp = Instant.ofEpochMilli(72); // Mocking is not easily compatible with annotation analysis, so we manually record // the method call. @@ -412,8 +414,10 @@ public void testLateDroppingForStatefulDoFnRunner() throws Exception { WindowedValue multiWindow = WindowedValues.of( 1, - new Instant(0), - ImmutableList.of(new IntervalWindow(new Instant(0), new Instant(0L + WINDOW_SIZE))), + Instant.ofEpochMilli(0), + ImmutableList.of( + new IntervalWindow( + Instant.ofEpochMilli(0), Instant.ofEpochMilli(0L + WINDOW_SIZE))), PaneInfo.ON_TIME_AND_ONLY_FIRING); runner.processElementInReadyWindows(multiWindow); @@ -432,7 +436,7 @@ public void testLateDroppingForStatefulDoFnRunner() throws Exception { @Test @Category({ValidatesRunner.class}) public void testGarbageCollectForStatefulDoFnRunner() throws Exception { - timerInternals.advanceInputWatermark(new Instant(1L)); + timerInternals.advanceInputWatermark(Instant.ofEpochMilli(1L)); MyDoFn fn = new MyDoFn(); StateTag> stateTag = StateTags.tagForSpec(fn.stateId, fn.intState); @@ -440,7 +444,7 @@ public void testGarbageCollectForStatefulDoFnRunner() throws Exception { PushbackSideInputDoFnRunner runner = createRunner(statefulRunner, ImmutableList.of(singletonView)); - Instant elementTime = new Instant(1); + Instant elementTime = Instant.ofEpochMilli(1); when(reader.isReady(Mockito.eq(singletonView), Mockito.any(BoundedWindow.class))) .thenReturn(true); diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/StateInternalsTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/StateInternalsTest.java index e15249969f26..8d788f0f72f4 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/StateInternalsTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/StateInternalsTest.java @@ -534,14 +534,14 @@ public void testWatermarkEarliestState() throws Exception { assertFalse(value.equals(underTest.state(NAMESPACE_2, WATERMARK_EARLIEST_ADDR))); assertThat(value.read(), Matchers.nullValue()); - value.add(new Instant(2000)); - assertThat(value.read(), equalTo(new Instant(2000))); + value.add(Instant.ofEpochMilli(2000)); + assertThat(value.read(), equalTo(Instant.ofEpochMilli(2000))); - value.add(new Instant(3000)); - assertThat(value.read(), equalTo(new Instant(2000))); + value.add(Instant.ofEpochMilli(3000)); + assertThat(value.read(), equalTo(Instant.ofEpochMilli(2000))); - value.add(new Instant(1000)); - assertThat(value.read(), equalTo(new Instant(1000))); + value.add(Instant.ofEpochMilli(1000)); + assertThat(value.read(), equalTo(Instant.ofEpochMilli(1000))); value.clear(); assertThat(value.read(), equalTo(null)); @@ -557,14 +557,14 @@ public void testWatermarkLatestState() throws Exception { assertFalse(value.equals(underTest.state(NAMESPACE_2, WATERMARK_LATEST_ADDR))); assertThat(value.read(), Matchers.nullValue()); - value.add(new Instant(2000)); - assertThat(value.read(), equalTo(new Instant(2000))); + value.add(Instant.ofEpochMilli(2000)); + assertThat(value.read(), equalTo(Instant.ofEpochMilli(2000))); - value.add(new Instant(3000)); - assertThat(value.read(), equalTo(new Instant(3000))); + value.add(Instant.ofEpochMilli(3000)); + assertThat(value.read(), equalTo(Instant.ofEpochMilli(3000))); - value.add(new Instant(1000)); - assertThat(value.read(), equalTo(new Instant(3000))); + value.add(Instant.ofEpochMilli(1000)); + assertThat(value.read(), equalTo(Instant.ofEpochMilli(3000))); value.clear(); assertThat(value.read(), equalTo(null)); @@ -580,8 +580,8 @@ public void testWatermarkEndOfWindowState() throws Exception { assertFalse(value.equals(underTest.state(NAMESPACE_2, WATERMARK_EOW_ADDR))); assertThat(value.read(), Matchers.nullValue()); - value.add(new Instant(2000)); - assertThat(value.read(), equalTo(new Instant(2000))); + value.add(Instant.ofEpochMilli(2000)); + assertThat(value.read(), equalTo(Instant.ofEpochMilli(2000))); value.clear(); assertThat(value.read(), equalTo(null)); @@ -594,7 +594,7 @@ public void testWatermarkStateIsEmpty() throws Exception { assertThat(value.isEmpty().read(), Matchers.is(true)); ReadableState readFuture = value.isEmpty(); - value.add(new Instant(1000)); + value.add(Instant.ofEpochMilli(1000)); assertThat(readFuture.read(), Matchers.is(false)); value.clear(); diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/StateNamespacesTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/StateNamespacesTest.java index 26e8c9e74d2f..6a994937ffb3 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/StateNamespacesTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/StateNamespacesTest.java @@ -37,7 +37,7 @@ public class StateNamespacesTest { private final Coder intervalCoder = IntervalWindow.getCoder(); private IntervalWindow intervalWindow(long start, long end) { - return new IntervalWindow(new Instant(start), new Instant(end)); + return new IntervalWindow(Instant.ofEpochMilli(start), Instant.ofEpochMilli(end)); } /** diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/StatefulDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/StatefulDoFnRunnerTest.java index df3aceff4a92..aecaf0629c74 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/StatefulDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/StatefulDoFnRunnerTest.java @@ -75,10 +75,10 @@ public class StatefulDoFnRunnerTest { .withAllowedLateness(Duration.millis(ALLOWED_LATENESS)); private static final IntervalWindow WINDOW_1 = - new IntervalWindow(new Instant(0), new Instant(10)); + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10)); private static final IntervalWindow WINDOW_2 = - new IntervalWindow(new Instant(10), new Instant(20)); + new IntervalWindow(Instant.ofEpochMilli(10), Instant.ofEpochMilli(20)); private final TupleTag outputTag = new TupleTag<>(); @@ -141,7 +141,7 @@ public void testOutputOrderedUnsupported() throws Exception { public void testDataDroppedBasedOnInputWatermarkWhenOrdered() throws Exception { MetricsContainerImpl container = new MetricsContainerImpl("any"); MetricsEnvironment.setCurrentContainer(container); - Instant timestamp = new Instant(0); + Instant timestamp = Instant.ofEpochMilli(0); MyDoFn fn = MyDoFn.create(true); @@ -192,8 +192,9 @@ private void testLateDropping(boolean ordered) throws Exception { runner.startBundle(); - IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(0L + WINDOW_SIZE)); - Instant timestamp = new Instant(0); + IntervalWindow window = + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(0L + WINDOW_SIZE)); + Instant timestamp = Instant.ofEpochMilli(0); runner.processElement( WindowedValues.of(KV.of("hello", 1), timestamp, window, PaneInfo.NO_FIRING)); @@ -210,14 +211,14 @@ private void testLateDropping(boolean ordered) throws Exception { } private void testGarbageCollect(boolean ordered) throws Exception { - timerInternals.advanceInputWatermark(new Instant(1L)); + timerInternals.advanceInputWatermark(Instant.ofEpochMilli(1L)); MyDoFn fn = MyDoFn.create(ordered); StateTag> stateTag = StateTags.tagForSpec(MyDoFn.STATE_ID, fn.intState()); DoFnRunner, Integer> runner = createStatefulDoFnRunner(fn); - Instant elementTime = new Instant(1); + Instant elementTime = Instant.ofEpochMilli(1); // first element, key is hello, WINDOW_1 runner.processElement( @@ -295,7 +296,7 @@ private void testOutput( runnerFactory) throws Exception { - timerInternals.advanceInputWatermark(new Instant(1L)); + timerInternals.advanceInputWatermark(Instant.ofEpochMilli(1L)); MyDoFn fn = MyDoFn.create(ordered); StateTag> stateTag = StateTags.tagForSpec(MyDoFn.STATE_ID, fn.intState()); @@ -304,7 +305,7 @@ private void testOutput( WindowedValueMultiReceiver output = asOutputManager(outputs); DoFnRunner, Integer> runner = runnerFactory.apply(fn, output); - Instant elementTime = new Instant(5); + Instant elementTime = Instant.ofEpochMilli(5); // write two elements, with descending timestamps runner.processElement( diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/TimerInternalsTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/TimerInternalsTest.java index 78f8322b8da4..ed1d834f978a 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/TimerInternalsTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/TimerInternalsTest.java @@ -45,8 +45,8 @@ public void testTimerDataCoder() throws Exception { TimerData.of( "arbitrary-id", StateNamespaces.global(), - new Instant(0), - new Instant(0), + Instant.ofEpochMilli(0), + Instant.ofEpochMilli(0), TimeDomain.EVENT_TIME, CausedByDrain.NORMAL)); @@ -56,9 +56,10 @@ public void testTimerDataCoder() throws Exception { TimerData.of( "another-id", StateNamespaces.window( - windowCoder, new IntervalWindow(new Instant(0), new Instant(100))), - new Instant(99), - new Instant(99), + windowCoder, + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(100))), + Instant.ofEpochMilli(99), + Instant.ofEpochMilli(99), TimeDomain.PROCESSING_TIME, CausedByDrain.NORMAL)); } @@ -70,7 +71,7 @@ public void testCoderIsSerializableWithWellKnownCoderType() { @Test public void testCompareEqual() { - Instant timestamp = new Instant(100); + Instant timestamp = Instant.ofEpochMilli(100); StateNamespace namespace = StateNamespaces.global(); TimerData timer = TimerData.of( @@ -90,8 +91,8 @@ public void testCompareEqual() { @Test public void testCompareByTimestamp() { - Instant firstTimestamp = new Instant(100); - Instant secondTimestamp = new Instant(200); + Instant firstTimestamp = Instant.ofEpochMilli(100); + Instant secondTimestamp = Instant.ofEpochMilli(200); StateNamespace namespace = StateNamespaces.global(); TimerData firstTimer = @@ -110,7 +111,7 @@ public void testCompareByTimestamp() { @Test public void testCompareByDomain() { - Instant timestamp = new Instant(100); + Instant timestamp = Instant.ofEpochMilli(100); StateNamespace namespace = StateNamespaces.global(); TimerData eventTimer = @@ -133,9 +134,9 @@ public void testCompareByDomain() { @Test public void testCompareByNamespace() { - Instant timestamp = new Instant(100); - IntervalWindow firstWindow = new IntervalWindow(new Instant(0), timestamp); - IntervalWindow secondWindow = new IntervalWindow(timestamp, new Instant(200)); + Instant timestamp = Instant.ofEpochMilli(100); + IntervalWindow firstWindow = new IntervalWindow(Instant.ofEpochMilli(0), timestamp); + IntervalWindow secondWindow = new IntervalWindow(timestamp, Instant.ofEpochMilli(200)); Coder windowCoder = IntervalWindow.getCoder(); StateNamespace firstWindowNs = StateNamespaces.window(windowCoder, firstWindow); @@ -153,7 +154,7 @@ public void testCompareByNamespace() { @Test public void testCompareByTimerId() { - Instant timestamp = new Instant(100); + Instant timestamp = Instant.ofEpochMilli(100); StateNamespace namespace = StateNamespaces.global(); TimerData id0Timer = diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/WindowMatchersTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/WindowMatchersTest.java index 06995a515fcf..3bfadf763be2 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/WindowMatchersTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/WindowMatchersTest.java @@ -42,13 +42,15 @@ public void testIsWindowedValueExact() { assertThat( WindowedValues.of( "hello", - new Instant(timestamp), - new IntervalWindow(new Instant(windowStart), new Instant(windowEnd)), + Instant.ofEpochMilli(timestamp), + new IntervalWindow(Instant.ofEpochMilli(windowStart), Instant.ofEpochMilli(windowEnd)), PaneInfo.NO_FIRING), WindowMatchers.isWindowedValue( "hello", - new Instant(timestamp), - ImmutableList.of(new IntervalWindow(new Instant(windowStart), new Instant(windowEnd))), + Instant.ofEpochMilli(timestamp), + ImmutableList.of( + new IntervalWindow( + Instant.ofEpochMilli(windowStart), Instant.ofEpochMilli(windowEnd))), PaneInfo.NO_FIRING)); } @@ -63,25 +65,29 @@ public void testIsWindowedValueReorderedWindows() { assertThat( WindowedValues.of( "hello", - new Instant(timestamp), + Instant.ofEpochMilli(timestamp), ImmutableList.of( - new IntervalWindow(new Instant(windowStart), new Instant(windowEnd)), - new IntervalWindow(new Instant(windowStart2), new Instant(windowEnd2))), + new IntervalWindow( + Instant.ofEpochMilli(windowStart), Instant.ofEpochMilli(windowEnd)), + new IntervalWindow( + Instant.ofEpochMilli(windowStart2), Instant.ofEpochMilli(windowEnd2))), PaneInfo.NO_FIRING), WindowMatchers.isWindowedValue( "hello", - new Instant(timestamp), + Instant.ofEpochMilli(timestamp), ImmutableList.of( - new IntervalWindow(new Instant(windowStart), new Instant(windowEnd)), - new IntervalWindow(new Instant(windowStart2), new Instant(windowEnd2))), + new IntervalWindow( + Instant.ofEpochMilli(windowStart), Instant.ofEpochMilli(windowEnd)), + new IntervalWindow( + Instant.ofEpochMilli(windowStart2), Instant.ofEpochMilli(windowEnd2))), PaneInfo.NO_FIRING)); } @Test public void test_IsValueInGlobalWindow_TimestampedValueInGlobalWindow() { assertThat( - WindowedValues.timestampedValueInGlobalWindow("foo", new Instant(7)), - WindowMatchers.isValueInGlobalWindow("foo", new Instant(7))); + WindowedValues.timestampedValueInGlobalWindow("foo", Instant.ofEpochMilli(7)), + WindowMatchers.isValueInGlobalWindow("foo", Instant.ofEpochMilli(7))); assertThat( WindowedValues.timestampedValueInGlobalWindow("foo", BoundedWindow.TIMESTAMP_MIN_VALUE), diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MonitoringInfoEncodingsTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MonitoringInfoEncodingsTest.java index 1f020c3a8ce1..c9532e2dc478 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MonitoringInfoEncodingsTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MonitoringInfoEncodingsTest.java @@ -81,7 +81,7 @@ public void testDoubleDistributionEncoding() { @Test public void testInt64GaugeEncoding() { - GaugeData data = GaugeData.create(1L, new Instant(2L)); + GaugeData data = GaugeData.create(1L, Instant.ofEpochMilli(2L)); ByteString payload = encodeInt64Gauge(data); assertEquals(ByteString.copyFrom(new byte[] {2, 1}), payload); assertEquals(data, decodeInt64Gauge(payload)); diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterAllStateMachineTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterAllStateMachineTest.java index b0874929fb63..33ef77c4c202 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterAllStateMachineTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterAllStateMachineTest.java @@ -46,7 +46,7 @@ public void testT1FiresFirst() throws Exception { AfterPaneStateMachine.elementCountAtLeast(2)), FixedWindows.of(Duration.millis(100))); - IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(100)); + IntervalWindow window = new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(100)); tester.injectElements(1); assertFalse(tester.shouldFire(window)); @@ -66,7 +66,7 @@ public void testT2FiresFirst() throws Exception { AfterPaneStateMachine.elementCountAtLeast(1)), FixedWindows.of(Duration.millis(100))); - IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(100)); + IntervalWindow window = new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(100)); tester.injectElements(1); assertFalse(tester.shouldFire(window)); @@ -93,20 +93,23 @@ public void testOnMergeRewinds() throws Exception { Sessions.withGapDuration(Duration.millis(10))); tester.injectElements(1); - IntervalWindow firstWindow = new IntervalWindow(new Instant(1), new Instant(11)); + IntervalWindow firstWindow = + new IntervalWindow(Instant.ofEpochMilli(1), Instant.ofEpochMilli(11)); tester.injectElements(5); - IntervalWindow secondWindow = new IntervalWindow(new Instant(5), new Instant(15)); + IntervalWindow secondWindow = + new IntervalWindow(Instant.ofEpochMilli(5), Instant.ofEpochMilli(15)); // Finish the AfterAll in the first window - tester.advanceInputWatermark(new Instant(11)); + tester.advanceInputWatermark(Instant.ofEpochMilli(11)); assertTrue(tester.shouldFire(firstWindow)); assertFalse(tester.shouldFire(secondWindow)); tester.fireIfShouldFire(firstWindow); // Merge them; the AfterAll should not be finished tester.mergeWindows(); - IntervalWindow mergedWindow = new IntervalWindow(new Instant(1), new Instant(15)); + IntervalWindow mergedWindow = + new IntervalWindow(Instant.ofEpochMilli(1), Instant.ofEpochMilli(15)); assertFalse(tester.isMarkedFinished(mergedWindow)); // Confirm that we are back on the first trigger by probing that it is not ready to fire @@ -116,7 +119,7 @@ public void testOnMergeRewinds() throws Exception { assertFalse(tester.shouldFire(mergedWindow)); // Fire the AfterAll in the merged window - tester.advanceInputWatermark(new Instant(15)); + tester.advanceInputWatermark(Instant.ofEpochMilli(15)); assertTrue(tester.shouldFire(mergedWindow)); tester.fireIfShouldFire(mergedWindow); diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterEachStateMachineTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterEachStateMachineTest.java index 301cf7d2c204..ed71b09f8d37 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterEachStateMachineTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterEachStateMachineTest.java @@ -58,7 +58,7 @@ public void testAfterEachInSequence() throws Exception { .orFinally(AfterWatermarkStateMachine.pastEndOfWindow())), FixedWindows.of(Duration.millis(10))); - IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(10)); + IntervalWindow window = new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10)); // AfterCount(2) not ready tester.injectElements(1); @@ -89,7 +89,7 @@ public void testAfterEachInSequence() throws Exception { assertFalse(tester.isMarkedFinished(window)); // This time advance the watermark to finish the whole mess. - tester.advanceInputWatermark(new Instant(10)); + tester.advanceInputWatermark(Instant.ofEpochMilli(10)); assertTrue(tester.shouldFire(window)); tester.fireIfShouldFire(window); assertTrue(tester.isMarkedFinished(window)); diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterFirstStateMachineTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterFirstStateMachineTest.java index 37855b24742e..cba2614c0e03 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterFirstStateMachineTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterFirstStateMachineTest.java @@ -60,7 +60,7 @@ public void testNeitherShouldFireFixedWindows() throws Exception { FixedWindows.of(Duration.millis(10))); tester.injectElements(1); - IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(10)); + IntervalWindow window = new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10)); when(mockTrigger1.shouldFire(anyTriggerContext())).thenReturn(false); when(mockTrigger2.shouldFire(anyTriggerContext())).thenReturn(false); @@ -76,7 +76,7 @@ public void testOnlyT1ShouldFireFixedWindows() throws Exception { AfterFirstStateMachine.of(mockTrigger1, mockTrigger2), FixedWindows.of(Duration.millis(10))); tester.injectElements(1); - IntervalWindow window = new IntervalWindow(new Instant(1), new Instant(11)); + IntervalWindow window = new IntervalWindow(Instant.ofEpochMilli(1), Instant.ofEpochMilli(11)); when(mockTrigger1.shouldFire(anyTriggerContext())).thenReturn(true); when(mockTrigger2.shouldFire(anyTriggerContext())).thenReturn(false); @@ -94,7 +94,7 @@ public void testOnlyT2ShouldFireFixedWindows() throws Exception { AfterFirstStateMachine.of(mockTrigger1, mockTrigger2), FixedWindows.of(Duration.millis(10))); tester.injectElements(1); - IntervalWindow window = new IntervalWindow(new Instant(1), new Instant(11)); + IntervalWindow window = new IntervalWindow(Instant.ofEpochMilli(1), Instant.ofEpochMilli(11)); when(mockTrigger1.shouldFire(anyTriggerContext())).thenReturn(false); when(mockTrigger2.shouldFire(anyTriggerContext())).thenReturn(true); @@ -111,7 +111,7 @@ public void testBothShouldFireFixedWindows() throws Exception { AfterFirstStateMachine.of(mockTrigger1, mockTrigger2), FixedWindows.of(Duration.millis(10))); tester.injectElements(1); - IntervalWindow window = new IntervalWindow(new Instant(1), new Instant(11)); + IntervalWindow window = new IntervalWindow(Instant.ofEpochMilli(1), Instant.ofEpochMilli(11)); when(mockTrigger1.shouldFire(anyTriggerContext())).thenReturn(true); when(mockTrigger2.shouldFire(anyTriggerContext())).thenReturn(true); @@ -138,20 +138,23 @@ public void testShouldFireAfterMerge() throws Exception { // Finished the AfterFirst in the first window tester.injectElements(1); - IntervalWindow firstWindow = new IntervalWindow(new Instant(1), new Instant(11)); + IntervalWindow firstWindow = + new IntervalWindow(Instant.ofEpochMilli(1), Instant.ofEpochMilli(11)); assertFalse(tester.shouldFire(firstWindow)); - tester.advanceInputWatermark(new Instant(11)); + tester.advanceInputWatermark(Instant.ofEpochMilli(11)); assertTrue(tester.shouldFire(firstWindow)); tester.fireIfShouldFire(firstWindow); // Set up second window where it is not done tester.injectElements(5); - IntervalWindow secondWindow = new IntervalWindow(new Instant(5), new Instant(15)); + IntervalWindow secondWindow = + new IntervalWindow(Instant.ofEpochMilli(5), Instant.ofEpochMilli(15)); assertFalse(tester.shouldFire(secondWindow)); // Merge them, if the merged window were on the second trigger, it would be ready tester.mergeWindows(); - IntervalWindow mergedWindow = new IntervalWindow(new Instant(1), new Instant(15)); + IntervalWindow mergedWindow = + new IntervalWindow(Instant.ofEpochMilli(1), Instant.ofEpochMilli(15)); assertFalse(tester.shouldFire(mergedWindow)); // Now adding 3 more makes the AfterFirst ready to fire diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterPaneStateMachineTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterPaneStateMachineTest.java index 8c1f947ef8eb..9e78282c75a7 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterPaneStateMachineTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterPaneStateMachineTest.java @@ -47,7 +47,7 @@ public void testAfterPaneElementCountFixedWindows() throws Exception { AfterPaneStateMachine.elementCountAtLeast(2), FixedWindows.of(Duration.millis(10))); tester.injectElements(1); // [0, 10) - IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(10)); + IntervalWindow window = new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10)); assertFalse(tester.shouldFire(window)); tester.injectElements(2); // [0, 10) @@ -58,7 +58,9 @@ public void testAfterPaneElementCountFixedWindows() throws Exception { assertTrue(tester.isMarkedFinished(window)); // But don't finish the other window - assertFalse(tester.isMarkedFinished(new IntervalWindow(new Instant(10), new Instant(20)))); + assertFalse( + tester.isMarkedFinished( + new IntervalWindow(Instant.ofEpochMilli(10), Instant.ofEpochMilli(20)))); } @Test @@ -68,7 +70,7 @@ public void testClear() throws Exception { AfterPaneStateMachine.elementCountAtLeast(2), FixedWindows.of(Duration.millis(10))); tester.injectElements(1, 2, 3); - IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(10)); + IntervalWindow window = new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10)); tester.clearState(window); tester.assertCleared(window); } @@ -84,12 +86,15 @@ public void testAfterPaneElementCountSessions() throws Exception { 1, // in [1, 11) 2); // in [2, 12) - assertFalse(tester.shouldFire(new IntervalWindow(new Instant(1), new Instant(11)))); - assertFalse(tester.shouldFire(new IntervalWindow(new Instant(2), new Instant(12)))); + assertFalse( + tester.shouldFire(new IntervalWindow(Instant.ofEpochMilli(1), Instant.ofEpochMilli(11)))); + assertFalse( + tester.shouldFire(new IntervalWindow(Instant.ofEpochMilli(2), Instant.ofEpochMilli(12)))); tester.mergeWindows(); - IntervalWindow mergedWindow = new IntervalWindow(new Instant(1), new Instant(12)); + IntervalWindow mergedWindow = + new IntervalWindow(Instant.ofEpochMilli(1), Instant.ofEpochMilli(12)); assertTrue(tester.shouldFire(mergedWindow)); tester.fireIfShouldFire(mergedWindow); assertTrue(tester.isMarkedFinished(mergedWindow)); @@ -102,7 +107,8 @@ public void testAfterPaneElementCountSessions() throws Exception { tester.mergeWindows(); - IntervalWindow newMergedWindow = new IntervalWindow(new Instant(7), new Instant(19)); + IntervalWindow newMergedWindow = + new IntervalWindow(Instant.ofEpochMilli(7), Instant.ofEpochMilli(19)); assertTrue(tester.shouldFire(newMergedWindow)); tester.fireIfShouldFire(newMergedWindow); assertTrue(tester.isMarkedFinished(newMergedWindow)); diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterProcessingTimeStateMachineTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterProcessingTimeStateMachineTest.java index 0c7490447cda..10bb67f3a3b2 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterProcessingTimeStateMachineTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterProcessingTimeStateMachineTest.java @@ -48,21 +48,23 @@ public void testAfterProcessingTimeFixedWindows() throws Exception { .plusDelayOf(Duration.millis(5)), FixedWindows.of(windowDuration)); - tester.advanceProcessingTime(new Instant(10)); + tester.advanceProcessingTime(Instant.ofEpochMilli(10)); // Timer at 15 tester.injectElements(1); - IntervalWindow firstWindow = new IntervalWindow(new Instant(0), new Instant(10)); - tester.advanceProcessingTime(new Instant(12)); + IntervalWindow firstWindow = + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10)); + tester.advanceProcessingTime(Instant.ofEpochMilli(12)); assertFalse(tester.shouldFire(firstWindow)); // Load up elements in the next window, timer at 17 for them tester.injectElements(11, 12, 13); - IntervalWindow secondWindow = new IntervalWindow(new Instant(10), new Instant(20)); + IntervalWindow secondWindow = + new IntervalWindow(Instant.ofEpochMilli(10), Instant.ofEpochMilli(20)); assertFalse(tester.shouldFire(secondWindow)); // Not quite time to fire - tester.advanceProcessingTime(new Instant(14)); + tester.advanceProcessingTime(Instant.ofEpochMilli(14)); assertFalse(tester.shouldFire(firstWindow)); assertFalse(tester.shouldFire(secondWindow)); @@ -70,14 +72,14 @@ public void testAfterProcessingTimeFixedWindows() throws Exception { tester.injectElements(2, 3); // Advance past the first timer and fire, finishing the first window - tester.advanceProcessingTime(new Instant(16)); + tester.advanceProcessingTime(Instant.ofEpochMilli(16)); assertTrue(tester.shouldFire(firstWindow)); assertFalse(tester.shouldFire(secondWindow)); tester.fireIfShouldFire(firstWindow); assertTrue(tester.isMarkedFinished(firstWindow)); // The next window fires and finishes now - tester.advanceProcessingTime(new Instant(18)); + tester.advanceProcessingTime(Instant.ofEpochMilli(18)); assertTrue(tester.shouldFire(secondWindow)); tester.fireIfShouldFire(secondWindow); assertTrue(tester.isMarkedFinished(secondWindow)); @@ -96,7 +98,7 @@ public void testClear() throws Exception { FixedWindows.of(Duration.millis(10))); tester.injectElements(1, 2, 3); - IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(10)); + IntervalWindow window = new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10)); tester.clearState(window); tester.assertCleared(window); } @@ -109,20 +111,23 @@ public void testAfterProcessingTimeWithMergingWindow() throws Exception { .plusDelayOf(Duration.millis(5)), Sessions.withGapDuration(Duration.millis(10))); - tester.advanceProcessingTime(new Instant(10)); + tester.advanceProcessingTime(Instant.ofEpochMilli(10)); tester.injectElements(1); // in [1, 11), timer for 15 - IntervalWindow firstWindow = new IntervalWindow(new Instant(1), new Instant(11)); + IntervalWindow firstWindow = + new IntervalWindow(Instant.ofEpochMilli(1), Instant.ofEpochMilli(11)); assertFalse(tester.shouldFire(firstWindow)); - tester.advanceProcessingTime(new Instant(12)); + tester.advanceProcessingTime(Instant.ofEpochMilli(12)); tester.injectElements(3); // in [3, 13), timer for 17 - IntervalWindow secondWindow = new IntervalWindow(new Instant(3), new Instant(13)); + IntervalWindow secondWindow = + new IntervalWindow(Instant.ofEpochMilli(3), Instant.ofEpochMilli(13)); assertFalse(tester.shouldFire(secondWindow)); tester.mergeWindows(); - IntervalWindow mergedWindow = new IntervalWindow(new Instant(1), new Instant(13)); + IntervalWindow mergedWindow = + new IntervalWindow(Instant.ofEpochMilli(1), Instant.ofEpochMilli(13)); - tester.advanceProcessingTime(new Instant(16)); + tester.advanceProcessingTime(Instant.ofEpochMilli(16)); assertTrue(tester.shouldFire(mergedWindow)); } diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterSynchronizedProcessingTimeStateMachineTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterSynchronizedProcessingTimeStateMachineTest.java index 1d4dc1499b84..da904b2ca50c 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterSynchronizedProcessingTimeStateMachineTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterSynchronizedProcessingTimeStateMachineTest.java @@ -43,21 +43,23 @@ public void testAfterProcessingTimeWithFixedWindows() throws Exception { .plusDelayOf(Duration.millis(5)), FixedWindows.of(windowDuration)); - tester.advanceProcessingTime(new Instant(10)); + tester.advanceProcessingTime(Instant.ofEpochMilli(10)); // Timer at 15 tester.injectElements(1); - IntervalWindow firstWindow = new IntervalWindow(new Instant(0), new Instant(10)); - tester.advanceProcessingTime(new Instant(12)); + IntervalWindow firstWindow = + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10)); + tester.advanceProcessingTime(Instant.ofEpochMilli(12)); assertFalse(tester.shouldFire(firstWindow)); // Load up elements in the next window, timer at 17 for them tester.injectElements(11, 12, 13); - IntervalWindow secondWindow = new IntervalWindow(new Instant(10), new Instant(20)); + IntervalWindow secondWindow = + new IntervalWindow(Instant.ofEpochMilli(10), Instant.ofEpochMilli(20)); assertFalse(tester.shouldFire(secondWindow)); // Not quite time to fire - tester.advanceProcessingTime(new Instant(14)); + tester.advanceProcessingTime(Instant.ofEpochMilli(14)); assertFalse(tester.shouldFire(firstWindow)); assertFalse(tester.shouldFire(secondWindow)); @@ -65,14 +67,14 @@ public void testAfterProcessingTimeWithFixedWindows() throws Exception { tester.injectElements(2, 3); // Advance past the first timer and fire, finishing the first window - tester.advanceProcessingTime(new Instant(16)); + tester.advanceProcessingTime(Instant.ofEpochMilli(16)); assertTrue(tester.shouldFire(firstWindow)); assertFalse(tester.shouldFire(secondWindow)); tester.fireIfShouldFire(firstWindow); assertTrue(tester.isMarkedFinished(firstWindow)); // The next window fires and finishes now - tester.advanceProcessingTime(new Instant(18)); + tester.advanceProcessingTime(Instant.ofEpochMilli(18)); assertTrue(tester.shouldFire(secondWindow)); tester.fireIfShouldFire(secondWindow); assertTrue(tester.isMarkedFinished(secondWindow)); @@ -87,20 +89,23 @@ public void testAfterProcessingTimeWithMergingWindow() throws Exception { .plusDelayOf(Duration.millis(5)), Sessions.withGapDuration(windowDuration)); - tester.advanceProcessingTime(new Instant(10)); + tester.advanceProcessingTime(Instant.ofEpochMilli(10)); tester.injectElements(1); // in [1, 11), timer for 15 - IntervalWindow firstWindow = new IntervalWindow(new Instant(1), new Instant(11)); + IntervalWindow firstWindow = + new IntervalWindow(Instant.ofEpochMilli(1), Instant.ofEpochMilli(11)); assertFalse(tester.shouldFire(firstWindow)); - tester.advanceProcessingTime(new Instant(12)); + tester.advanceProcessingTime(Instant.ofEpochMilli(12)); tester.injectElements(3); // in [3, 13), timer for 17 - IntervalWindow secondWindow = new IntervalWindow(new Instant(3), new Instant(13)); + IntervalWindow secondWindow = + new IntervalWindow(Instant.ofEpochMilli(3), Instant.ofEpochMilli(13)); assertFalse(tester.shouldFire(secondWindow)); tester.mergeWindows(); - IntervalWindow mergedWindow = new IntervalWindow(new Instant(1), new Instant(13)); + IntervalWindow mergedWindow = + new IntervalWindow(Instant.ofEpochMilli(1), Instant.ofEpochMilli(13)); - tester.advanceProcessingTime(new Instant(16)); + tester.advanceProcessingTime(Instant.ofEpochMilli(16)); assertTrue(tester.shouldFire(mergedWindow)); } } diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterWatermarkStateMachineTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterWatermarkStateMachineTest.java index c71e85c57118..c24e511045fb 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterWatermarkStateMachineTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterWatermarkStateMachineTest.java @@ -95,13 +95,13 @@ public void testEarlyAndAtWatermark() throws Exception { FixedWindows.of(Duration.millis(100))); injectElements(1); - IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(100)); + IntervalWindow window = new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(100)); testRunningAsTrigger(mockEarly, window); // Fire due to watermark when(mockEarly.shouldFire(anyTriggerContext())).thenReturn(false); - tester.advanceInputWatermark(new Instant(100)); + tester.advanceInputWatermark(Instant.ofEpochMilli(100)); assertTrue(tester.shouldFire(window)); tester.fireIfShouldFire(window); assertTrue(tester.isMarkedFinished(window)); @@ -115,7 +115,7 @@ public void testTimerForEndOfWindow() throws Exception { assertThat(tester.getNextTimer(TimeDomain.EVENT_TIME), nullValue()); injectElements(1); - IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(100)); + IntervalWindow window = new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(100)); assertThat(tester.getNextTimer(TimeDomain.EVENT_TIME), equalTo(window.maxTimestamp())); } @@ -128,7 +128,7 @@ public void testTimerForEndOfWindowCompound() throws Exception { assertThat(tester.getNextTimer(TimeDomain.EVENT_TIME), nullValue()); injectElements(1); - IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(100)); + IntervalWindow window = new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(100)); assertThat(tester.getNextTimer(TimeDomain.EVENT_TIME), equalTo(window.maxTimestamp())); } @@ -140,7 +140,7 @@ public void testAtWatermarkAndLate() throws Exception { FixedWindows.of(Duration.millis(100))); injectElements(1); - IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(100)); + IntervalWindow window = new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(100)); // No early firing, just double checking when(mockEarly.shouldFire(anyTriggerContext())).thenReturn(true); @@ -150,7 +150,7 @@ public void testAtWatermarkAndLate() throws Exception { // Fire due to watermark when(mockEarly.shouldFire(anyTriggerContext())).thenReturn(false); - tester.advanceInputWatermark(new Instant(100)); + tester.advanceInputWatermark(Instant.ofEpochMilli(100)); assertTrue(tester.shouldFire(window)); tester.fireIfShouldFire(window); assertFalse(tester.isMarkedFinished(window)); @@ -168,13 +168,13 @@ public void testEarlyAndAtWatermarkAndLate() throws Exception { FixedWindows.of(Duration.millis(100))); injectElements(1); - IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(100)); + IntervalWindow window = new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(100)); testRunningAsTrigger(mockEarly, window); // Fire due to watermark when(mockEarly.shouldFire(anyTriggerContext())).thenReturn(false); - tester.advanceInputWatermark(new Instant(100)); + tester.advanceInputWatermark(Instant.ofEpochMilli(100)); assertTrue(tester.shouldFire(window)); tester.fireIfShouldFire(window); assertFalse(tester.isMarkedFinished(window)); @@ -201,12 +201,15 @@ public void testOnMergeAlreadyFinished() throws Exception { tester.injectElements(1); tester.injectElements(5); - IntervalWindow firstWindow = new IntervalWindow(new Instant(1), new Instant(11)); - IntervalWindow secondWindow = new IntervalWindow(new Instant(5), new Instant(15)); - IntervalWindow mergedWindow = new IntervalWindow(new Instant(1), new Instant(15)); + IntervalWindow firstWindow = + new IntervalWindow(Instant.ofEpochMilli(1), Instant.ofEpochMilli(11)); + IntervalWindow secondWindow = + new IntervalWindow(Instant.ofEpochMilli(5), Instant.ofEpochMilli(15)); + IntervalWindow mergedWindow = + new IntervalWindow(Instant.ofEpochMilli(1), Instant.ofEpochMilli(15)); // Finish the AfterWatermark.pastEndOfWindow() trigger in both windows - tester.advanceInputWatermark(new Instant(15)); + tester.advanceInputWatermark(Instant.ofEpochMilli(15)); assertTrue(tester.shouldFire(firstWindow)); assertTrue(tester.shouldFire(secondWindow)); tester.fireIfShouldFire(firstWindow); @@ -249,12 +252,15 @@ public void testOnMergeRewinds() throws Exception { tester.injectElements(1); tester.injectElements(5); - IntervalWindow firstWindow = new IntervalWindow(new Instant(1), new Instant(11)); - IntervalWindow secondWindow = new IntervalWindow(new Instant(5), new Instant(15)); - IntervalWindow mergedWindow = new IntervalWindow(new Instant(1), new Instant(15)); + IntervalWindow firstWindow = + new IntervalWindow(Instant.ofEpochMilli(1), Instant.ofEpochMilli(11)); + IntervalWindow secondWindow = + new IntervalWindow(Instant.ofEpochMilli(5), Instant.ofEpochMilli(15)); + IntervalWindow mergedWindow = + new IntervalWindow(Instant.ofEpochMilli(1), Instant.ofEpochMilli(15)); // Finish the AfterWatermark.pastEndOfWindow() trigger in only the first window - tester.advanceInputWatermark(new Instant(11)); + tester.advanceInputWatermark(Instant.ofEpochMilli(11)); assertTrue(tester.shouldFire(firstWindow)); assertFalse(tester.shouldFire(secondWindow)); tester.fireIfShouldFire(firstWindow); @@ -274,7 +280,7 @@ public void testOnMergeRewinds() throws Exception { assertFalse(tester.shouldFire(mergedWindow)); // And confirm that advancing the watermark fires again - tester.advanceInputWatermark(new Instant(15)); + tester.advanceInputWatermark(Instant.ofEpochMilli(15)); assertTrue(tester.shouldFire(mergedWindow)); } @@ -297,12 +303,15 @@ public void testEarlyAndLateOnMergeAlreadyFinished() throws Exception { tester.injectElements(1); tester.injectElements(5); - IntervalWindow firstWindow = new IntervalWindow(new Instant(1), new Instant(11)); - IntervalWindow secondWindow = new IntervalWindow(new Instant(5), new Instant(15)); - IntervalWindow mergedWindow = new IntervalWindow(new Instant(1), new Instant(15)); + IntervalWindow firstWindow = + new IntervalWindow(Instant.ofEpochMilli(1), Instant.ofEpochMilli(11)); + IntervalWindow secondWindow = + new IntervalWindow(Instant.ofEpochMilli(5), Instant.ofEpochMilli(15)); + IntervalWindow mergedWindow = + new IntervalWindow(Instant.ofEpochMilli(1), Instant.ofEpochMilli(15)); // Finish the AfterWatermark.pastEndOfWindow() bit of the trigger in both windows - tester.advanceInputWatermark(new Instant(15)); + tester.advanceInputWatermark(Instant.ofEpochMilli(15)); assertTrue(tester.shouldFire(firstWindow)); assertTrue(tester.shouldFire(secondWindow)); tester.fireIfShouldFire(firstWindow); @@ -364,12 +373,15 @@ public void testEarlyAndLateOnMergeRewinds() throws Exception { tester.injectElements(1); tester.injectElements(5); - IntervalWindow firstWindow = new IntervalWindow(new Instant(1), new Instant(11)); - IntervalWindow secondWindow = new IntervalWindow(new Instant(5), new Instant(15)); - IntervalWindow mergedWindow = new IntervalWindow(new Instant(1), new Instant(15)); + IntervalWindow firstWindow = + new IntervalWindow(Instant.ofEpochMilli(1), Instant.ofEpochMilli(11)); + IntervalWindow secondWindow = + new IntervalWindow(Instant.ofEpochMilli(5), Instant.ofEpochMilli(15)); + IntervalWindow mergedWindow = + new IntervalWindow(Instant.ofEpochMilli(1), Instant.ofEpochMilli(15)); // Finish the AfterWatermark.pastEndOfWindow() bit of the trigger in only the first window - tester.advanceInputWatermark(new Instant(11)); + tester.advanceInputWatermark(Instant.ofEpochMilli(11)); assertTrue(tester.shouldFire(firstWindow)); assertFalse(tester.shouldFire(secondWindow)); tester.fireIfShouldFire(firstWindow); @@ -389,7 +401,7 @@ public void testEarlyAndLateOnMergeRewinds() throws Exception { assertFalse(tester.shouldFire(mergedWindow)); // And confirm that advancing the watermark fires again - tester.advanceInputWatermark(new Instant(15)); + tester.advanceInputWatermark(Instant.ofEpochMilli(15)); assertTrue(tester.shouldFire(mergedWindow)); } diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/DefaultTriggerStateMachineTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/DefaultTriggerStateMachineTest.java index f0dc347017d3..e20a605765ec 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/DefaultTriggerStateMachineTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/DefaultTriggerStateMachineTest.java @@ -50,16 +50,18 @@ public void testDefaultTriggerFixedWindows() throws Exception { 1, // [0, 100) 101); // [100, 200) - IntervalWindow firstWindow = new IntervalWindow(new Instant(0), new Instant(100)); - IntervalWindow secondWindow = new IntervalWindow(new Instant(100), new Instant(200)); + IntervalWindow firstWindow = + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(100)); + IntervalWindow secondWindow = + new IntervalWindow(Instant.ofEpochMilli(100), Instant.ofEpochMilli(200)); // Advance the watermark almost to the end of the first window. - tester.advanceInputWatermark(new Instant(99)); + tester.advanceInputWatermark(Instant.ofEpochMilli(99)); assertFalse(tester.shouldFire(firstWindow)); assertFalse(tester.shouldFire(secondWindow)); // Advance watermark past end of the first window, which is then ready - tester.advanceInputWatermark(new Instant(100)); + tester.advanceInputWatermark(Instant.ofEpochMilli(100)); assertTrue(tester.shouldFire(firstWindow)); assertFalse(tester.shouldFire(secondWindow)); @@ -69,7 +71,7 @@ public void testDefaultTriggerFixedWindows() throws Exception { assertFalse(tester.shouldFire(secondWindow)); // Advance watermark to 200, then both are ready - tester.advanceInputWatermark(new Instant(200)); + tester.advanceInputWatermark(Instant.ofEpochMilli(200)); assertTrue(tester.shouldFire(firstWindow)); assertTrue(tester.shouldFire(secondWindow)); @@ -88,16 +90,19 @@ public void testDefaultTriggerSlidingWindows() throws Exception { 1, // [-50, 50), [0, 100) 50); // [0, 100), [50, 150) - IntervalWindow firstWindow = new IntervalWindow(new Instant(-50), new Instant(50)); - IntervalWindow secondWindow = new IntervalWindow(new Instant(0), new Instant(100)); - IntervalWindow thirdWindow = new IntervalWindow(new Instant(50), new Instant(150)); + IntervalWindow firstWindow = + new IntervalWindow(Instant.ofEpochMilli(-50), Instant.ofEpochMilli(50)); + IntervalWindow secondWindow = + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(100)); + IntervalWindow thirdWindow = + new IntervalWindow(Instant.ofEpochMilli(50), Instant.ofEpochMilli(150)); assertFalse(tester.shouldFire(firstWindow)); assertFalse(tester.shouldFire(secondWindow)); assertFalse(tester.shouldFire(thirdWindow)); // At 50, the first becomes ready; it stays ready after firing - tester.advanceInputWatermark(new Instant(50)); + tester.advanceInputWatermark(Instant.ofEpochMilli(50)); assertTrue(tester.shouldFire(firstWindow)); assertFalse(tester.shouldFire(secondWindow)); assertFalse(tester.shouldFire(thirdWindow)); @@ -107,13 +112,13 @@ public void testDefaultTriggerSlidingWindows() throws Exception { assertFalse(tester.shouldFire(thirdWindow)); // At 99, the first is still the only one ready - tester.advanceInputWatermark(new Instant(99)); + tester.advanceInputWatermark(Instant.ofEpochMilli(99)); assertTrue(tester.shouldFire(firstWindow)); assertFalse(tester.shouldFire(secondWindow)); assertFalse(tester.shouldFire(thirdWindow)); // At 100, the first and second are ready - tester.advanceInputWatermark(new Instant(100)); + tester.advanceInputWatermark(Instant.ofEpochMilli(100)); assertTrue(tester.shouldFire(firstWindow)); assertTrue(tester.shouldFire(secondWindow)); assertFalse(tester.shouldFire(thirdWindow)); @@ -135,24 +140,27 @@ public void testDefaultTriggerSessions() throws Exception { 50); // [50, 150) tester.mergeWindows(); - IntervalWindow firstWindow = new IntervalWindow(new Instant(1), new Instant(101)); - IntervalWindow secondWindow = new IntervalWindow(new Instant(50), new Instant(150)); - IntervalWindow mergedWindow = new IntervalWindow(new Instant(1), new Instant(150)); + IntervalWindow firstWindow = + new IntervalWindow(Instant.ofEpochMilli(1), Instant.ofEpochMilli(101)); + IntervalWindow secondWindow = + new IntervalWindow(Instant.ofEpochMilli(50), Instant.ofEpochMilli(150)); + IntervalWindow mergedWindow = + new IntervalWindow(Instant.ofEpochMilli(1), Instant.ofEpochMilli(150)); // Not ready in any window yet - tester.advanceInputWatermark(new Instant(100)); + tester.advanceInputWatermark(Instant.ofEpochMilli(100)); assertFalse(tester.shouldFire(firstWindow)); assertFalse(tester.shouldFire(secondWindow)); assertFalse(tester.shouldFire(mergedWindow)); // The first window is "ready": the caller owns knowledge of which windows are merged away - tester.advanceInputWatermark(new Instant(149)); + tester.advanceInputWatermark(Instant.ofEpochMilli(149)); assertTrue(tester.shouldFire(firstWindow)); assertFalse(tester.shouldFire(secondWindow)); assertFalse(tester.shouldFire(mergedWindow)); // Now ready on all windows - tester.advanceInputWatermark(new Instant(150)); + tester.advanceInputWatermark(Instant.ofEpochMilli(150)); assertTrue(tester.shouldFire(firstWindow)); assertTrue(tester.shouldFire(secondWindow)); assertTrue(tester.shouldFire(mergedWindow)); diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/NeverStateMachineTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/NeverStateMachineTest.java index 384aa12639fa..6108ea4f1bd1 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/NeverStateMachineTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/NeverStateMachineTest.java @@ -46,9 +46,10 @@ public void setup() throws Exception { @Test public void falseAfterEndOfWindow() throws Exception { - triggerTester.injectElements(TimestampedValue.of(1, new Instant(1))); + triggerTester.injectElements(TimestampedValue.of(1, Instant.ofEpochMilli(1))); IntervalWindow window = - new IntervalWindow(new Instant(0), new Instant(0).plus(Duration.standardMinutes(5))); + new IntervalWindow( + Instant.ofEpochMilli(0), Instant.ofEpochMilli(0).plus(Duration.standardMinutes(5))); assertThat(triggerTester.shouldFire(window), is(false)); triggerTester.advanceInputWatermark(BoundedWindow.TIMESTAMP_MAX_VALUE); assertThat(triggerTester.shouldFire(window), is(false)); diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/OrFinallyStateMachineTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/OrFinallyStateMachineTest.java index dcf0b96be39f..332519b3b7fc 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/OrFinallyStateMachineTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/OrFinallyStateMachineTest.java @@ -50,7 +50,7 @@ public void testActualFiresAndFinishes() throws Exception { AfterPaneStateMachine.elementCountAtLeast(100)), FixedWindows.of(Duration.millis(100))); - IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(100)); + IntervalWindow window = new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(100)); // Not yet firing tester.injectElements(1); @@ -77,7 +77,7 @@ public void testActualFiresOnly() throws Exception { AfterPaneStateMachine.elementCountAtLeast(100)), FixedWindows.of(Duration.millis(100))); - IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(100)); + IntervalWindow window = new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(100)); // Not yet firing tester.injectElements(1); @@ -113,20 +113,23 @@ public void testShouldFireAfterMerge() throws Exception { // Finished the orFinally in the first window tester.injectElements(1); - IntervalWindow firstWindow = new IntervalWindow(new Instant(1), new Instant(11)); + IntervalWindow firstWindow = + new IntervalWindow(Instant.ofEpochMilli(1), Instant.ofEpochMilli(11)); assertFalse(tester.shouldFire(firstWindow)); - tester.advanceInputWatermark(new Instant(11)); + tester.advanceInputWatermark(Instant.ofEpochMilli(11)); assertTrue(tester.shouldFire(firstWindow)); tester.fireIfShouldFire(firstWindow); // Set up second window where it is not done tester.injectElements(5); - IntervalWindow secondWindow = new IntervalWindow(new Instant(5), new Instant(15)); + IntervalWindow secondWindow = + new IntervalWindow(Instant.ofEpochMilli(5), Instant.ofEpochMilli(15)); assertFalse(tester.shouldFire(secondWindow)); // Merge them, if the merged window were on the second trigger, it would be ready tester.mergeWindows(); - IntervalWindow mergedWindow = new IntervalWindow(new Instant(1), new Instant(15)); + IntervalWindow mergedWindow = + new IntervalWindow(Instant.ofEpochMilli(1), Instant.ofEpochMilli(15)); assertFalse(tester.shouldFire(mergedWindow)); // Now adding 3 more makes the main trigger ready to fire @@ -148,7 +151,7 @@ public void testActualFiresButUntilFinishes() throws Exception { AfterPaneStateMachine.elementCountAtLeast(3)), FixedWindows.of(Duration.millis(10))); - IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(10)); + IntervalWindow window = new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10)); // Before any firing tester.injectElements(1); diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/RepeatedlyStateMachineTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/RepeatedlyStateMachineTest.java index 08312360806e..e96786b17c34 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/RepeatedlyStateMachineTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/RepeatedlyStateMachineTest.java @@ -70,10 +70,12 @@ public void testShouldFire() throws Exception { setUp(FixedWindows.of(Duration.millis(10))); when(mockTrigger.shouldFire(anyTriggerContext())).thenReturn(true); - assertTrue(tester.shouldFire(new IntervalWindow(new Instant(0), new Instant(10)))); + assertTrue( + tester.shouldFire(new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10)))); when(mockTrigger.shouldFire(Mockito.any())).thenReturn(false); - assertFalse(tester.shouldFire(new IntervalWindow(new Instant(0), new Instant(10)))); + assertFalse( + tester.shouldFire(new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10)))); } @Test @@ -84,16 +86,19 @@ public void testShouldFireAfterMerge() throws Exception { Sessions.withGapDuration(Duration.millis(10))); tester.injectElements(1); - IntervalWindow firstWindow = new IntervalWindow(new Instant(1), new Instant(11)); + IntervalWindow firstWindow = + new IntervalWindow(Instant.ofEpochMilli(1), Instant.ofEpochMilli(11)); assertFalse(tester.shouldFire(firstWindow)); tester.injectElements(5); - IntervalWindow secondWindow = new IntervalWindow(new Instant(5), new Instant(15)); + IntervalWindow secondWindow = + new IntervalWindow(Instant.ofEpochMilli(5), Instant.ofEpochMilli(15)); assertFalse(tester.shouldFire(secondWindow)); // Merge them, if the merged window were on the second trigger, it would be ready tester.mergeWindows(); - IntervalWindow mergedWindow = new IntervalWindow(new Instant(1), new Instant(15)); + IntervalWindow mergedWindow = + new IntervalWindow(Instant.ofEpochMilli(1), Instant.ofEpochMilli(15)); assertTrue(tester.shouldFire(mergedWindow)); } @@ -135,7 +140,7 @@ public void testRepeatedlyAfterFirstProcessingTime() throws Exception { tester.injectElements(1); assertFalse(tester.shouldFire(window)); - tester.advanceProcessingTime(new Instant(0).plus(Duration.standardMinutes(15))); + tester.advanceProcessingTime(Instant.ofEpochMilli(0).plus(Duration.standardMinutes(15))); assertTrue(tester.shouldFire(window)); tester.fireIfShouldFire(window); assertFalse(tester.shouldFire(window)); @@ -173,7 +178,7 @@ public void testRepeatedlyProcessingTime() throws Exception { tester.injectElements(1); assertFalse(tester.shouldFire(window)); - tester.advanceProcessingTime(new Instant(0).plus(Duration.standardMinutes(15))); + tester.advanceProcessingTime(Instant.ofEpochMilli(0).plus(Duration.standardMinutes(15))); assertTrue(tester.shouldFire(window)); tester.fireIfShouldFire(window); assertFalse(tester.shouldFire(window)); diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/ReshuffleTriggerStateMachineTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/ReshuffleTriggerStateMachineTest.java index 191efc33c572..88742df7a6c1 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/ReshuffleTriggerStateMachineTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/ReshuffleTriggerStateMachineTest.java @@ -43,7 +43,8 @@ public void testShouldFire() throws Exception { TriggerStateMachineTester tester = TriggerStateMachineTester.forTrigger( ReshuffleTriggerStateMachine.create(), FixedWindows.of(Duration.millis(100))); - IntervalWindow arbitraryWindow = new IntervalWindow(new Instant(300), new Instant(400)); + IntervalWindow arbitraryWindow = + new IntervalWindow(Instant.ofEpochMilli(300), Instant.ofEpochMilli(400)); assertTrue(tester.shouldFire(arbitraryWindow)); } @@ -52,7 +53,8 @@ public void testOnTimer() throws Exception { TriggerStateMachineTester tester = TriggerStateMachineTester.forTrigger( ReshuffleTriggerStateMachine.create(), FixedWindows.of(Duration.millis(100))); - IntervalWindow arbitraryWindow = new IntervalWindow(new Instant(100), new Instant(200)); + IntervalWindow arbitraryWindow = + new IntervalWindow(Instant.ofEpochMilli(100), Instant.ofEpochMilli(200)); tester.fireIfShouldFire(arbitraryWindow); assertFalse(tester.isMarkedFinished(arbitraryWindow)); } diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java index 0f3f17c337ce..392b158e8733 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java @@ -86,7 +86,7 @@ public void injectElements(int... values) throws Exception { List> timestampedValues = Lists.newArrayListWithCapacity(values.length); for (int value : values) { - timestampedValues.add(TimestampedValue.of(value, new Instant(value))); + timestampedValues.add(TimestampedValue.of(value, Instant.ofEpochMilli(value))); } injectElements(timestampedValues); } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java index 95cadef7afdb..f26b53d1cc4d 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java @@ -251,7 +251,7 @@ private boolean isKeyed(PValue pvalue) { public State waitUntilFinish(Duration duration) throws Exception { Instant completionTime; if (duration.equals(Duration.ZERO)) { - completionTime = new Instant(Long.MAX_VALUE); + completionTime = Instant.ofEpochMilli(Long.MAX_VALUE); } else { completionTime = Instant.now().plus(duration); } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/NanosOffsetClock.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/NanosOffsetClock.java index f26c907541fa..98a57fe7560e 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/NanosOffsetClock.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/NanosOffsetClock.java @@ -36,7 +36,7 @@ private NanosOffsetClock() { @Override public Instant now() { - return new Instant( + return Instant.ofEpochMilli( baseMillis + TimeUnit.MILLISECONDS.convert( System.nanoTime() - nanosAtBaseMillis, TimeUnit.NANOSECONDS)); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternalsTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternalsTest.java index ae2de1dc67f1..7f5bf1afc928 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternalsTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternalsTest.java @@ -263,20 +263,20 @@ public void testWatermarkHoldStateWithUnderlying() { WatermarkHoldState underlyingValue = underlying.state(namespace, stateTag); assertThat(underlyingValue.read(), nullValue()); - underlyingValue.add(new Instant(250L)); - assertThat(underlyingValue.read(), equalTo(new Instant(250L))); + underlyingValue.add(Instant.ofEpochMilli(250L)); + assertThat(underlyingValue.read(), equalTo(Instant.ofEpochMilli(250L))); CopyOnAccessInMemoryStateInternals internals = CopyOnAccessInMemoryStateInternals.withUnderlying(key, underlying); WatermarkHoldState copyOnAccessState = internals.state(namespace, stateTag); - assertThat(copyOnAccessState.read(), equalTo(new Instant(250L))); + assertThat(copyOnAccessState.read(), equalTo(Instant.ofEpochMilli(250L))); - copyOnAccessState.add(new Instant(100L)); - assertThat(copyOnAccessState.read(), equalTo(new Instant(100L))); - assertThat(underlyingValue.read(), equalTo(new Instant(250L))); + copyOnAccessState.add(Instant.ofEpochMilli(100L)); + assertThat(copyOnAccessState.read(), equalTo(Instant.ofEpochMilli(100L))); + assertThat(underlyingValue.read(), equalTo(Instant.ofEpochMilli(250L))); - copyOnAccessState.add(new Instant(500L)); - assertThat(copyOnAccessState.read(), equalTo(new Instant(100L))); + copyOnAccessState.add(Instant.ofEpochMilli(500L)); + assertThat(copyOnAccessState.read(), equalTo(Instant.ofEpochMilli(100L))); WatermarkHoldState reReadUnderlyingValue = underlying.state(namespace, stateTag); assertThat(underlyingValue.read(), equalTo(reReadUnderlyingValue.read())); @@ -467,14 +467,14 @@ public void testGetEarliestWatermarkHoldAfterCommit() { new BoundedWindow() { @Override public Instant maxTimestamp() { - return new Instant(2048L); + return Instant.ofEpochMilli(2048L); } }; BoundedWindow second = new BoundedWindow() { @Override public Instant maxTimestamp() { - return new Instant(689743L); + return Instant.ofEpochMilli(689743L); } }; CopyOnAccessInMemoryStateInternals internals = @@ -484,16 +484,16 @@ public Instant maxTimestamp() { StateTags.watermarkStateInternal("foo", TimestampCombiner.EARLIEST); WatermarkHoldState firstHold = internals.state(StateNamespaces.window(null, first), firstHoldAddress); - firstHold.add(new Instant(22L)); + firstHold.add(Instant.ofEpochMilli(22L)); StateTag secondHoldAddress = StateTags.watermarkStateInternal("foo", TimestampCombiner.EARLIEST); WatermarkHoldState secondHold = internals.state(StateNamespaces.window(null, second), secondHoldAddress); - secondHold.add(new Instant(2L)); + secondHold.add(Instant.ofEpochMilli(2L)); internals.commit(); - assertThat(internals.getEarliestWatermarkHold(), equalTo(new Instant(2L))); + assertThat(internals.getEarliestWatermarkHold(), equalTo(Instant.ofEpochMilli(2L))); } @Test @@ -502,14 +502,14 @@ public void testGetEarliestWatermarkHoldWithEarliestInUnderlyingTable() { new BoundedWindow() { @Override public Instant maxTimestamp() { - return new Instant(2048L); + return Instant.ofEpochMilli(2048L); } }; BoundedWindow second = new BoundedWindow() { @Override public Instant maxTimestamp() { - return new Instant(689743L); + return Instant.ofEpochMilli(689743L); } }; CopyOnAccessInMemoryStateInternals underlying = @@ -518,7 +518,7 @@ public Instant maxTimestamp() { StateTags.watermarkStateInternal("foo", TimestampCombiner.EARLIEST); WatermarkHoldState firstHold = underlying.state(StateNamespaces.window(null, first), firstHoldAddress); - firstHold.add(new Instant(22L)); + firstHold.add(Instant.ofEpochMilli(22L)); CopyOnAccessInMemoryStateInternals internals = CopyOnAccessInMemoryStateInternals.withUnderlying("foo", underlying.commit()); @@ -527,10 +527,10 @@ public Instant maxTimestamp() { StateTags.watermarkStateInternal("foo", TimestampCombiner.EARLIEST); WatermarkHoldState secondHold = internals.state(StateNamespaces.window(null, second), secondHoldAddress); - secondHold.add(new Instant(244L)); + secondHold.add(Instant.ofEpochMilli(244L)); internals.commit(); - assertThat(internals.getEarliestWatermarkHold(), equalTo(new Instant(22L))); + assertThat(internals.getEarliestWatermarkHold(), equalTo(Instant.ofEpochMilli(22L))); } @Test @@ -539,14 +539,14 @@ public void testGetEarliestWatermarkHoldWithEarliestInNewTable() { new BoundedWindow() { @Override public Instant maxTimestamp() { - return new Instant(2048L); + return Instant.ofEpochMilli(2048L); } }; BoundedWindow second = new BoundedWindow() { @Override public Instant maxTimestamp() { - return new Instant(689743L); + return Instant.ofEpochMilli(689743L); } }; CopyOnAccessInMemoryStateInternals underlying = @@ -555,7 +555,7 @@ public Instant maxTimestamp() { StateTags.watermarkStateInternal("foo", TimestampCombiner.EARLIEST); WatermarkHoldState firstHold = underlying.state(StateNamespaces.window(null, first), firstHoldAddress); - firstHold.add(new Instant(224L)); + firstHold.add(Instant.ofEpochMilli(224L)); CopyOnAccessInMemoryStateInternals internals = CopyOnAccessInMemoryStateInternals.withUnderlying("foo", underlying.commit()); @@ -564,10 +564,10 @@ public Instant maxTimestamp() { StateTags.watermarkStateInternal("foo", TimestampCombiner.EARLIEST); WatermarkHoldState secondHold = internals.state(StateNamespaces.window(null, second), secondHoldAddress); - secondHold.add(new Instant(24L)); + secondHold.add(Instant.ofEpochMilli(24L)); internals.commit(); - assertThat(internals.getEarliestWatermarkHold(), equalTo(new Instant(24L))); + assertThat(internals.getEarliestWatermarkHold(), equalTo(Instant.ofEpochMilli(24L))); } @Test @@ -579,7 +579,7 @@ public void testGetEarliestHoldBeforeCommit() { .state( StateNamespaces.global(), StateTags.watermarkStateInternal("foo", TimestampCombiner.EARLIEST)) - .add(new Instant(1234L)); + .add(Instant.ofEpochMilli(1234L)); thrown.expect(IllegalStateException.class); thrown.expectMessage(CopyOnAccessInMemoryStateInternals.class.getSimpleName()); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectTimerInternalsTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectTimerInternalsTest.java index d8a68b8ccea6..56c36ae25881 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectTimerInternalsTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectTimerInternalsTest.java @@ -56,7 +56,7 @@ public class DirectTimerInternalsTest { @Before public void setup() { MockitoAnnotations.initMocks(this); - clock = MockClock.fromInstant(new Instant(0)); + clock = MockClock.fromInstant(Instant.ofEpochMilli(0)); timerUpdateBuilder = TimerUpdate.builder(StructuralKey.of(1234, VarIntCoder.of())); @@ -68,22 +68,22 @@ public void setTimerAddsToBuilder() { TimerData eventTimer = TimerData.of( StateNamespaces.global(), - new Instant(20145L), - new Instant(20145L), + Instant.ofEpochMilli(20145L), + Instant.ofEpochMilli(20145L), TimeDomain.EVENT_TIME, CausedByDrain.NORMAL); TimerData processingTimer = TimerData.of( StateNamespaces.global(), - new Instant(125555555L), - new Instant(125555555L), + Instant.ofEpochMilli(125555555L), + Instant.ofEpochMilli(125555555L), TimeDomain.PROCESSING_TIME, CausedByDrain.NORMAL); TimerData synchronizedProcessingTimer = TimerData.of( StateNamespaces.global(), - new Instant(98745632189L), - new Instant(98745632189L), + Instant.ofEpochMilli(98745632189L), + Instant.ofEpochMilli(98745632189L), TimeDomain.SYNCHRONIZED_PROCESSING_TIME, CausedByDrain.NORMAL); internals.setTimer(eventTimer); @@ -100,22 +100,22 @@ public void deleteTimerDeletesOnBuilder() { TimerData eventTimer = TimerData.of( StateNamespaces.global(), - new Instant(20145L), - new Instant(20145L), + Instant.ofEpochMilli(20145L), + Instant.ofEpochMilli(20145L), TimeDomain.EVENT_TIME, CausedByDrain.NORMAL); TimerData processingTimer = TimerData.of( StateNamespaces.global(), - new Instant(125555555L), - new Instant(125555555L), + Instant.ofEpochMilli(125555555L), + Instant.ofEpochMilli(125555555L), TimeDomain.PROCESSING_TIME, CausedByDrain.NORMAL); TimerData synchronizedProcessingTimer = TimerData.of( StateNamespaces.global(), - new Instant(98745632189L), - new Instant(98745632189L), + Instant.ofEpochMilli(98745632189L), + Instant.ofEpochMilli(98745632189L), TimeDomain.SYNCHRONIZED_PROCESSING_TIME, CausedByDrain.NORMAL); internals.deleteTimer(eventTimer); @@ -142,19 +142,20 @@ public void getProcessingTimeIsClockNow() { @Test public void getSynchronizedProcessingTimeIsWatermarkSynchronizedInputTime() { - when(watermarks.getSynchronizedProcessingInputTime()).thenReturn(new Instant(12345L)); - assertThat(internals.currentSynchronizedProcessingTime(), equalTo(new Instant(12345L))); + when(watermarks.getSynchronizedProcessingInputTime()).thenReturn(Instant.ofEpochMilli(12345L)); + assertThat( + internals.currentSynchronizedProcessingTime(), equalTo(Instant.ofEpochMilli(12345L))); } @Test public void getInputWatermarkTimeUsesWatermarkTime() { - when(watermarks.getInputWatermark()).thenReturn(new Instant(8765L)); - assertThat(internals.currentInputWatermarkTime(), equalTo(new Instant(8765L))); + when(watermarks.getInputWatermark()).thenReturn(Instant.ofEpochMilli(8765L)); + assertThat(internals.currentInputWatermarkTime(), equalTo(Instant.ofEpochMilli(8765L))); } @Test public void getOutputWatermarkTimeUsesWatermarkTime() { - when(watermarks.getOutputWatermark()).thenReturn(new Instant(25525L)); - assertThat(internals.currentOutputWatermarkTime(), equalTo(new Instant(25525L))); + when(watermarks.getOutputWatermark()).thenReturn(Instant.ofEpochMilli(25525L)); + assertThat(internals.currentOutputWatermarkTime(), equalTo(Instant.ofEpochMilli(25525L))); } } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluatorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluatorTest.java index 53c02e796002..7f4c2fc86ed3 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluatorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluatorTest.java @@ -109,8 +109,8 @@ public void removesOnExceptionInOnTimer() throws Exception { TimerData.of( "foo", StateNamespaces.global(), - new Instant(0), - new Instant(0), + Instant.ofEpochMilli(0), + Instant.ofEpochMilli(0), TimeDomain.EVENT_TIME, CausedByDrain.NORMAL), "", diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java index 3375ebfa36ce..9c1acb2404b3 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java @@ -149,21 +149,24 @@ public void writeToViewWriterThenReadReads() { IsBounded.BOUNDED, IterableCoder.of(KvCoder.of(VoidCoder.of(), VarIntCoder.of()))), view); - BoundedWindow window = new TestBoundedWindow(new Instant(1024L)); - BoundedWindow second = new TestBoundedWindow(new Instant(899999L)); + BoundedWindow window = new TestBoundedWindow(Instant.ofEpochMilli(1024L)); + BoundedWindow second = new TestBoundedWindow(Instant.ofEpochMilli(899999L)); ImmutableList.Builder> valuesBuilder = ImmutableList.builder(); for (Object materializedValue : materializeValuesFor(view.getPipeline().getOptions(), View.asIterable(), 1)) { valuesBuilder.add( WindowedValues.of( - materializedValue, new Instant(1222), window, PaneInfo.ON_TIME_AND_ONLY_FIRING)); + materializedValue, + Instant.ofEpochMilli(1222), + window, + PaneInfo.ON_TIME_AND_ONLY_FIRING)); } for (Object materializedValue : materializeValuesFor(view.getPipeline().getOptions(), View.asIterable(), 2)) { valuesBuilder.add( WindowedValues.of( materializedValue, - new Instant(8766L), + Instant.ofEpochMilli(8766L), second, PaneInfo.createPane(true, false, Timing.ON_TIME, 0, 0))); } @@ -179,7 +182,7 @@ public void writeToViewWriterThenReadReads() { overwrittenValuesBuilder.add( WindowedValues.of( materializedValue, - new Instant(8677L), + Instant.ofEpochMilli(8677L), second, PaneInfo.createPane(false, true, Timing.LATE, 1, 1))); } @@ -299,7 +302,7 @@ public void callAfterOutputMustHaveBeenProducedAfterEndOfWatermarkCallsback() th downstream, GlobalWindow.INSTANCE, WindowingStrategy.globalDefault(), callback); TransformResult result = - StepTransformResult.withHold(impulseProducer, new Instant(0)).build(); + StepTransformResult.withHold(impulseProducer, Instant.ofEpochMilli(0)).build(); context.handleResult(null, ImmutableList.of(), result); // Difficult to demonstrate that we took no action in a multithreaded world; poll for a bit @@ -329,15 +332,15 @@ public void callAfterOutputMustHaveBeenProducedAlreadyAfterCallsImmediately() th @Test public void extractFiredTimersExtractsTimers() { TransformResult holdResult = - StepTransformResult.withHold(impulseProducer, new Instant(0)).build(); + StepTransformResult.withHold(impulseProducer, Instant.ofEpochMilli(0)).build(); context.handleResult(null, ImmutableList.of(), holdResult); StructuralKey key = StructuralKey.of("foo".length(), VarIntCoder.of()); TimerData toFire = TimerData.of( StateNamespaces.global(), - new Instant(100L), - new Instant(100L), + Instant.ofEpochMilli(100L), + Instant.ofEpochMilli(100L), TimeDomain.EVENT_TIME, CausedByDrain.NORMAL); TransformResult timerResult = diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java index a8097d4898f1..d9e5544a04a3 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java @@ -77,12 +77,12 @@ public void testFlattenInMemoryEvaluator() throws Exception { leftSideEvaluator.processElement(WindowedValues.valueInGlobalWindow(1)); rightSideEvaluator.processElement(WindowedValues.valueInGlobalWindow(-1)); leftSideEvaluator.processElement( - WindowedValues.timestampedValueInGlobalWindow(2, new Instant(1024))); + WindowedValues.timestampedValueInGlobalWindow(2, Instant.ofEpochMilli(1024))); leftSideEvaluator.processElement(WindowedValues.valueInGlobalWindow(4, PaneInfo.NO_FIRING)); rightSideEvaluator.processElement( WindowedValues.valueInGlobalWindow(2, PaneInfo.ON_TIME_AND_ONLY_FIRING)); rightSideEvaluator.processElement( - WindowedValues.timestampedValueInGlobalWindow(-4, new Instant(-4096))); + WindowedValues.timestampedValueInGlobalWindow(-4, Instant.ofEpochMilli(-4096))); TransformResult rightSideResult = rightSideEvaluator.finishBundle(); TransformResult leftSideResult = leftSideEvaluator.finishBundle(); @@ -99,14 +99,14 @@ public void testFlattenInMemoryEvaluator() throws Exception { assertThat( flattenedLeftBundle.commit(Instant.now()).getElements(), containsInAnyOrder( - WindowedValues.timestampedValueInGlobalWindow(2, new Instant(1024)), + WindowedValues.timestampedValueInGlobalWindow(2, Instant.ofEpochMilli(1024)), WindowedValues.valueInGlobalWindow(4, PaneInfo.NO_FIRING), WindowedValues.valueInGlobalWindow(1))); assertThat( flattenedRightBundle.commit(Instant.now()).getElements(), containsInAnyOrder( WindowedValues.valueInGlobalWindow(2, PaneInfo.ON_TIME_AND_ONLY_FIRING), - WindowedValues.timestampedValueInGlobalWindow(-4, new Instant(-4096)), + WindowedValues.timestampedValueInGlobalWindow(-4, Instant.ofEpochMilli(-4096)), WindowedValues.valueInGlobalWindow(-1))); } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java index bdfbb04d78c8..84cc37395340 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java @@ -81,8 +81,8 @@ public void noMutationKeyedBundleSucceeds() { WindowedValue windowedArray = WindowedValues.of( new byte[] {4, 8, 12}, - new Instant(891L), - new IntervalWindow(new Instant(0), new Instant(1000)), + Instant.ofEpochMilli(891L), + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(1000)), PaneInfo.ON_TIME_AND_ONLY_FIRING); keyed.add(windowedArray); @@ -97,8 +97,8 @@ public void noMutationCreateBundleSucceeds() { WindowedValue windowedArray = WindowedValues.of( new byte[] {4, 8, 12}, - new Instant(891L), - new IntervalWindow(new Instant(0), new Instant(1000)), + Instant.ofEpochMilli(891L), + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(1000)), PaneInfo.ON_TIME_AND_ONLY_FIRING); intermediate.add(windowedArray); @@ -116,8 +116,8 @@ public void mutationBeforeAddKeyedBundleSucceeds() { WindowedValue windowedArray = WindowedValues.of( array, - new Instant(891L), - new IntervalWindow(new Instant(0), new Instant(1000)), + Instant.ofEpochMilli(891L), + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(1000)), PaneInfo.ON_TIME_AND_ONLY_FIRING); keyed.add(windowedArray); @@ -133,8 +133,8 @@ public void mutationBeforeAddCreateBundleSucceeds() { WindowedValue windowedArray = WindowedValues.of( array, - new Instant(891L), - new IntervalWindow(new Instant(0), new Instant(1000)), + Instant.ofEpochMilli(891L), + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(1000)), PaneInfo.ON_TIME_AND_ONLY_FIRING); array[2] = -3; intermediate.add(windowedArray); @@ -152,8 +152,8 @@ public void mutationAfterAddKeyedBundleThrows() { WindowedValue windowedArray = WindowedValues.of( array, - new Instant(891L), - new IntervalWindow(new Instant(0), new Instant(1000)), + Instant.ofEpochMilli(891L), + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(1000)), PaneInfo.ON_TIME_AND_ONLY_FIRING); keyed.add(windowedArray); @@ -171,8 +171,8 @@ public void mutationAfterAddCreateBundleThrows() { WindowedValue windowedArray = WindowedValues.of( array, - new Instant(891L), - new IntervalWindow(new Instant(0), new Instant(1000)), + Instant.ofEpochMilli(891L), + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(1000)), PaneInfo.ON_TIME_AND_ONLY_FIRING); intermediate.add(windowedArray); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutableListBundleFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutableListBundleFactoryTest.java index 6b84908d626e..fe9aa8178997 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutableListBundleFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutableListBundleFactoryTest.java @@ -134,7 +134,7 @@ public void getElementsBeforeAddShouldReturnEmptyIterable() { public void getElementsAfterAddShouldReturnAddedElements() { WindowedValue firstValue = WindowedValues.valueInGlobalWindow(1); WindowedValue secondValue = - WindowedValues.timestampedValueInGlobalWindow(2, new Instant(1000L)); + WindowedValues.timestampedValueInGlobalWindow(2, Instant.ofEpochMilli(1000L)); afterCommitGetElementsShouldHaveAddedElements(ImmutableList.of(firstValue, secondValue)); } @@ -166,7 +166,7 @@ public void addElementsPastEndOfTimeThrows() { public void withElementsShouldReturnIndependentBundle() { WindowedValue firstValue = WindowedValues.valueInGlobalWindow(1); WindowedValue secondValue = - WindowedValues.timestampedValueInGlobalWindow(2, new Instant(1000L)); + WindowedValues.timestampedValueInGlobalWindow(2, Instant.ofEpochMilli(1000L)); CommittedBundle committed = afterCommitGetElementsShouldHaveAddedElements(ImmutableList.of(firstValue, secondValue)); @@ -174,8 +174,8 @@ public void withElementsShouldReturnIndependentBundle() { WindowedValue firstReplacement = WindowedValues.of( 9, - new Instant(2048L), - new IntervalWindow(new Instant(2044L), Instant.now()), + Instant.ofEpochMilli(2048L), + new IntervalWindow(Instant.ofEpochMilli(2044L), Instant.now()), PaneInfo.NO_FIRING); WindowedValue secondReplacement = WindowedValues.timestampedValueInGlobalWindow(-1, Instant.now()); @@ -189,7 +189,7 @@ public void withElementsShouldReturnIndependentBundle() { assertThat( withed.getSynchronizedProcessingOutputWatermark(), equalTo(committed.getSynchronizedProcessingOutputWatermark())); - assertThat(withed.getMinimumTimestamp(), equalTo(new Instant(2048L))); + assertThat(withed.getMinimumTimestamp(), equalTo(Instant.ofEpochMilli(2048L))); } @Test diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java index 95338e4bf330..9a2d54589c2a 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java @@ -144,8 +144,8 @@ public void keyedInputWithKeyPreserving() { "hello", WindowedValues.of( KV.of("hello", 3), - new Instant(0), - new IntervalWindow(new Instant(0), new Instant(9)), + Instant.ofEpochMilli(0), + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(9)), PaneInfo.NO_FIRING))) .withCoder( KvCoder.of( diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/MultiStepCombineTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/MultiStepCombineTest.java index 8a99358ece02..e5bad1dda2c9 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/MultiStepCombineTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/MultiStepCombineTest.java @@ -83,22 +83,22 @@ public void testMultiStepCombineWindowed() { pipeline .apply( Create.timestamped( - TimestampedValue.of(KV.of("foo", 1L), new Instant(1L)), - TimestampedValue.of(KV.of("bar", 2L), new Instant(2L)), - TimestampedValue.of(KV.of("bizzle", 3L), new Instant(3L)), - TimestampedValue.of(KV.of("bar", 4L), new Instant(4L)), - TimestampedValue.of(KV.of("bizzle", 11L), new Instant(11L)))) + TimestampedValue.of(KV.of("foo", 1L), Instant.ofEpochMilli(1L)), + TimestampedValue.of(KV.of("bar", 2L), Instant.ofEpochMilli(2L)), + TimestampedValue.of(KV.of("bizzle", 3L), Instant.ofEpochMilli(3L)), + TimestampedValue.of(KV.of("bar", 4L), Instant.ofEpochMilli(4L)), + TimestampedValue.of(KV.of("bizzle", 11L), Instant.ofEpochMilli(11L)))) .apply(Window.into(windowFn)) .apply(Combine.perKey(new MultiStepCombineFn())); PAssert.that("Windows should combine only elements in their windows", combined) - .inWindow(new IntervalWindow(new Instant(0L), Duration.millis(6L))) + .inWindow(new IntervalWindow(Instant.ofEpochMilli(0L), Duration.millis(6L))) .containsInAnyOrder(KV.of("foo", 1L), KV.of("bar", 6L), KV.of("bizzle", 3L)); PAssert.that("Elements should appear in all the windows they are assigned to", combined) - .inWindow(new IntervalWindow(new Instant(-3L), Duration.millis(6L))) + .inWindow(new IntervalWindow(Instant.ofEpochMilli(-3L), Duration.millis(6L))) .containsInAnyOrder(KV.of("foo", 1L), KV.of("bar", 2L)); PAssert.that(combined) - .inWindow(new IntervalWindow(new Instant(6L), Duration.millis(6L))) + .inWindow(new IntervalWindow(Instant.ofEpochMilli(6L), Duration.millis(6L))) .containsInAnyOrder(KV.of("bizzle", 11L)); PAssert.that(combined) .containsInAnyOrder( @@ -121,10 +121,10 @@ public void testMultiStepCombineTimestampCombiner() { pipeline .apply( Create.timestamped( - TimestampedValue.of(KV.of("foo", 4L), new Instant(1L)), - TimestampedValue.of(KV.of("foo", 1L), new Instant(4L)), - TimestampedValue.of(KV.of("bazzle", 4L), new Instant(4L)), - TimestampedValue.of(KV.of("foo", 12L), new Instant(12L)))) + TimestampedValue.of(KV.of("foo", 4L), Instant.ofEpochMilli(1L)), + TimestampedValue.of(KV.of("foo", 1L), Instant.ofEpochMilli(4L)), + TimestampedValue.of(KV.of("bazzle", 4L), Instant.ofEpochMilli(4L)), + TimestampedValue.of(KV.of("foo", 12L), Instant.ofEpochMilli(12L)))) .apply( Window.>into(FixedWindows.of(Duration.millis(5L))) .withTimestampCombiner(combiner)) @@ -145,9 +145,9 @@ public void reifyTimestamp(ProcessContext context) { PAssert.that(reified) .containsInAnyOrder( - KV.of("foo", TimestampedValue.of(5L, new Instant(4L))), - KV.of("bazzle", TimestampedValue.of(4L, new Instant(4L))), - KV.of("foo", TimestampedValue.of(12L, new Instant(12L)))); + KV.of("foo", TimestampedValue.of(5L, Instant.ofEpochMilli(4L))), + KV.of("bazzle", TimestampedValue.of(4L, Instant.ofEpochMilli(4L))), + KV.of("foo", TimestampedValue.of(12L, Instant.ofEpochMilli(12L)))); pipeline.run(); } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java index 4429385d7975..736e03c86724 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java @@ -102,14 +102,15 @@ public void sideInputsNotReadyResultHasUnprocessedElements() { ParDoEvaluator evaluator = createEvaluator(singletonView, fn, inputPc, output); - IntervalWindow nonGlobalWindow = new IntervalWindow(new Instant(0), new Instant(10_000L)); + IntervalWindow nonGlobalWindow = + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10_000L)); WindowedValue first = WindowedValues.valueInGlobalWindow(3); WindowedValue second = - WindowedValues.of(2, new Instant(1234L), nonGlobalWindow, PaneInfo.NO_FIRING); + WindowedValues.of(2, Instant.ofEpochMilli(1234L), nonGlobalWindow, PaneInfo.NO_FIRING); WindowedValue third = WindowedValues.of( 1, - new Instant(2468L), + Instant.ofEpochMilli(2468L), ImmutableList.of(nonGlobalWindow, GlobalWindow.INSTANCE), PaneInfo.NO_FIRING); @@ -121,14 +122,16 @@ public void sideInputsNotReadyResultHasUnprocessedElements() { assertThat( result.getUnprocessedElements(), Matchers.>containsInAnyOrder( - second, WindowedValues.of(1, new Instant(2468L), nonGlobalWindow, PaneInfo.NO_FIRING))); + second, + WindowedValues.of( + 1, Instant.ofEpochMilli(2468L), nonGlobalWindow, PaneInfo.NO_FIRING))); assertThat(result.getOutputBundles(), Matchers.contains(outputBundle)); assertThat(fn.processed, containsInAnyOrder(1, 3)); assertThat( Iterables.getOnlyElement(result.getOutputBundles()).commit(Instant.now()).getElements(), containsInAnyOrder( first.withValue(8), - WindowedValues.timestampedValueInGlobalWindow(6, new Instant(2468L)))); + WindowedValues.timestampedValueInGlobalWindow(6, Instant.ofEpochMilli(2468L)))); } private ParDoEvaluator createEvaluator( diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java index 2f5b48192716..b64d5762bb29 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java @@ -70,7 +70,7 @@ public class SideInputContainerTest { new BoundedWindow() { @Override public Instant maxTimestamp() { - return new Instant(789541L); + return Instant.ofEpochMilli(789541L); } @Override @@ -83,7 +83,7 @@ public String toString() { new BoundedWindow() { @Override public Instant maxTimestamp() { - return new Instant(14564786L); + return Instant.ofEpochMilli(14564786L); } @Override @@ -128,13 +128,19 @@ public void getAfterWriteReturnsPaneInWindow() throws Exception { materializeValuesFor(mapView.getPipeline().getOptions(), View.asMap(), KV.of("one", 1))) { valuesBuilder.add( WindowedValues.of( - materializedValue, new Instant(1L), FIRST_WINDOW, PaneInfo.ON_TIME_AND_ONLY_FIRING)); + materializedValue, + Instant.ofEpochMilli(1L), + FIRST_WINDOW, + PaneInfo.ON_TIME_AND_ONLY_FIRING)); } for (Object materializedValue : materializeValuesFor(mapView.getPipeline().getOptions(), View.asMap(), KV.of("two", 2))) { valuesBuilder.add( WindowedValues.of( - materializedValue, new Instant(20L), FIRST_WINDOW, PaneInfo.ON_TIME_AND_ONLY_FIRING)); + materializedValue, + Instant.ofEpochMilli(20L), + FIRST_WINDOW, + PaneInfo.ON_TIME_AND_ONLY_FIRING)); } container.write(mapView, valuesBuilder.build()); @@ -153,7 +159,7 @@ public void getReturnsLatestPaneInWindow() throws Exception { valuesBuilder.add( WindowedValues.of( materializedValue, - new Instant(1L), + Instant.ofEpochMilli(1L), SECOND_WINDOW, PaneInfo.createPane(true, false, Timing.EARLY))); } @@ -162,7 +168,7 @@ public void getReturnsLatestPaneInWindow() throws Exception { valuesBuilder.add( WindowedValues.of( materializedValue, - new Instant(20L), + Instant.ofEpochMilli(20L), SECOND_WINDOW, PaneInfo.createPane(true, false, Timing.EARLY))); } @@ -180,7 +186,7 @@ public void getReturnsLatestPaneInWindow() throws Exception { overwriteValuesBuilder.add( WindowedValues.of( materializedValue, - new Instant(300L), + Instant.ofEpochMilli(300L), SECOND_WINDOW, PaneInfo.createPane(false, false, Timing.EARLY, 1, -1))); } @@ -315,7 +321,7 @@ public void finishDoesNotOverwriteWrittenElements() throws Exception { valuesBuilder.add( WindowedValues.of( materializedValue, - new Instant(1L), + Instant.ofEpochMilli(1L), SECOND_WINDOW, PaneInfo.createPane(true, false, Timing.EARLY))); } @@ -324,7 +330,7 @@ public void finishDoesNotOverwriteWrittenElements() throws Exception { valuesBuilder.add( WindowedValues.of( materializedValue, - new Instant(20L), + Instant.ofEpochMilli(20L), SECOND_WINDOW, PaneInfo.createPane(true, false, Timing.EARLY))); } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java index cd3f0ef925e4..d9f1050cfc63 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java @@ -197,14 +197,15 @@ public void process(ProcessContext c) {} when(mockSideInputReader.isReady(ArgumentMatchers.any(), ArgumentMatchers.any())) .thenReturn(false); - IntervalWindow firstWindow = new IntervalWindow(new Instant(0), new Instant(9)); + IntervalWindow firstWindow = + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(9)); // A single bundle with some elements in the global window; it should register cleanup for the // global window state merely by having the evaluator created. The cleanup logic does not // depend on the window. String key = "hello"; WindowedValue> firstKv = - WindowedValues.of(KV.of(key, 1), new Instant(3), firstWindow, PaneInfo.NO_FIRING); + WindowedValues.of(KV.of(key, 1), Instant.ofEpochMilli(3), firstWindow, PaneInfo.NO_FIRING); WindowedValue>> gbkOutputElement = firstKv.withValue( diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java index 164aad503ff9..da114a588ee9 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java @@ -74,7 +74,7 @@ public void producesElementsInSequence() throws Exception { TestStream testStream = TestStream.create(VarIntCoder.of()) .addElements(1, 2, 3) - .advanceWatermarkTo(new Instant(0)) + .advanceWatermarkTo(Instant.ofEpochMilli(0)) .addElements( TimestampedValue.atMinimumTimestamp(4), TimestampedValue.atMinimumTimestamp(5), @@ -118,7 +118,7 @@ public void producesElementsInSequence() throws Exception { (WindowedValue>) Iterables.getOnlyElement(secondResult.getUnprocessedElements()); assertThat(secondResidual.getValue().getIndex(), equalTo(2)); - assertThat(secondResidual.getTimestamp(), equalTo(new Instant(0))); + assertThat(secondResidual.getTimestamp(), equalTo(Instant.ofEpochMilli(0))); CommittedBundle> thirdBundle = secondBundle.withElements(Collections.singleton(secondResidual)); @@ -131,7 +131,7 @@ public void producesElementsInSequence() throws Exception { (WindowedValue>) Iterables.getOnlyElement(thirdResult.getUnprocessedElements()); assertThat(thirdResidual.getValue().getIndex(), equalTo(3)); - assertThat(thirdResidual.getTimestamp(), equalTo(new Instant(0))); + assertThat(thirdResidual.getTimestamp(), equalTo(Instant.ofEpochMilli(0))); Instant start = clock.now(); CommittedBundle> fourthBundle = @@ -146,7 +146,7 @@ public void producesElementsInSequence() throws Exception { (WindowedValue>) Iterables.getOnlyElement(fourthResult.getUnprocessedElements()); assertThat(fourthResidual.getValue().getIndex(), equalTo(4)); - assertThat(fourthResidual.getTimestamp(), equalTo(new Instant(0))); + assertThat(fourthResidual.getTimestamp(), equalTo(Instant.ofEpochMilli(0))); CommittedBundle> fifthBundle = thirdBundle.withElements(Collections.singleton(fourthResidual)); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java index 5413a694e92b..697177193282 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java @@ -481,13 +481,13 @@ private void processElement(final TestUnboundedSource source) throws Exc * is the epoch offset by the value of the element. */ private static WindowedValue tgw(Long elem) { - return WindowedValues.timestampedValueInGlobalWindow(elem, new Instant(elem)); + return WindowedValues.timestampedValueInGlobalWindow(elem, Instant.ofEpochMilli(elem)); } private static class LongToInstantFn implements SerializableFunction { @Override public Instant apply(Long input) { - return new Instant(input); + return Instant.ofEpochMilli(input); } } @@ -582,7 +582,7 @@ public Instant getWatermark() { if (index + 1 == elems.size() && TestUnboundedSource.this.advanceWatermarkToInfinity) { return BoundedWindow.TIMESTAMP_MAX_VALUE; } else { - return new Instant(index + getWatermarkCalls); + return Instant.ofEpochMilli(index + getWatermarkCalls); } } @@ -603,7 +603,7 @@ public T getCurrent() throws NoSuchElementException { @Override public Instant getCurrentTimestamp() throws NoSuchElementException { - return new Instant(index); + return Instant.ofEpochMilli(index); } @Override diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkCallbackExecutorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkCallbackExecutorTest.java index 19d592e63b49..e99a1a66f790 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkCallbackExecutorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkCallbackExecutorTest.java @@ -80,13 +80,14 @@ public void multipleCallbacksShouldFireFires() throws Exception { CountDownLatch latch = new CountDownLatch(2); WindowFn windowFn = FixedWindows.of(Duration.standardMinutes(10)); IntervalWindow window = - new IntervalWindow(new Instant(0L), new Instant(0L).plus(Duration.standardMinutes(10))); + new IntervalWindow( + Instant.ofEpochMilli(0L), Instant.ofEpochMilli(0L).plus(Duration.standardMinutes(10))); executor.callOnGuaranteedFiring( create, window, WindowingStrategy.of(windowFn), new CountDownLatchCallback(latch)); executor.callOnGuaranteedFiring( create, window, WindowingStrategy.of(windowFn), new CountDownLatchCallback(latch)); - executor.fireForWatermark(create, new Instant(0L).plus(Duration.standardMinutes(10))); + executor.fireForWatermark(create, Instant.ofEpochMilli(0L).plus(Duration.standardMinutes(10))); assertThat(latch.await(500, TimeUnit.MILLISECONDS), equalTo(true)); } @@ -95,11 +96,12 @@ public void noCallbacksShouldFire() throws Exception { CountDownLatch latch = new CountDownLatch(1); WindowFn windowFn = FixedWindows.of(Duration.standardMinutes(10)); IntervalWindow window = - new IntervalWindow(new Instant(0L), new Instant(0L).plus(Duration.standardMinutes(10))); + new IntervalWindow( + Instant.ofEpochMilli(0L), Instant.ofEpochMilli(0L).plus(Duration.standardMinutes(10))); executor.callOnGuaranteedFiring( create, window, WindowingStrategy.of(windowFn), new CountDownLatchCallback(latch)); - executor.fireForWatermark(create, new Instant(0L).plus(Duration.standardMinutes(5))); + executor.fireForWatermark(create, Instant.ofEpochMilli(0L).plus(Duration.standardMinutes(5))); assertThat(latch.await(500, TimeUnit.MILLISECONDS), equalTo(false)); } @@ -108,11 +110,12 @@ public void unrelatedStepShouldNotFire() throws Exception { CountDownLatch latch = new CountDownLatch(1); WindowFn windowFn = FixedWindows.of(Duration.standardMinutes(10)); IntervalWindow window = - new IntervalWindow(new Instant(0L), new Instant(0L).plus(Duration.standardMinutes(10))); + new IntervalWindow( + Instant.ofEpochMilli(0L), Instant.ofEpochMilli(0L).plus(Duration.standardMinutes(10))); executor.callOnGuaranteedFiring( sum, window, WindowingStrategy.of(windowFn), new CountDownLatchCallback(latch)); - executor.fireForWatermark(create, new Instant(0L).plus(Duration.standardMinutes(20))); + executor.fireForWatermark(create, Instant.ofEpochMilli(0L).plus(Duration.standardMinutes(20))); assertThat(latch.await(500, TimeUnit.MILLISECONDS), equalTo(false)); } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java index 1853344ee1ac..9700638c7cf3 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java @@ -137,7 +137,7 @@ public void processElement(ProcessContext c) throws Exception { PCollectionList preFlatten = PCollectionList.of(impulse).and(impulseToFlatten); flattened = preFlatten.apply("flattened", Flatten.pCollections()); - clock = MockClock.fromInstant(new Instant(1000)); + clock = MockClock.fromInstant(Instant.ofEpochMilli(1000)); DirectGraphs.performDirectOverrides(p); graph = DirectGraphs.getGraph(p); @@ -170,11 +170,11 @@ public void getWatermarkForUpdatedSourceTransform() { graph.getProducer(impulse), null, Collections.singleton(output), - new Instant(8000L)); + Instant.ofEpochMilli(8000L)); manager.refreshAll(); TransformWatermarks updatedSourceWatermark = manager.getWatermarks(graph.getProducer(impulse)); - assertThat(updatedSourceWatermark.getOutputWatermark(), equalTo(new Instant(8000L))); + assertThat(updatedSourceWatermark.getOutputWatermark(), equalTo(Instant.ofEpochMilli(8000L))); } /** @@ -246,7 +246,7 @@ public void getWatermarkForMultiInputTransform() { transformAfterProcessing.getOutputWatermark(), not(greaterThan(BoundedWindow.TIMESTAMP_MIN_VALUE))); - Instant firstCollectionTimestamp = new Instant(10000); + Instant firstCollectionTimestamp = Instant.ofEpochMilli(10000); CommittedBundle firstPcollectionBundle = timestampedBundle(impulse, TimestampedValue.of(new byte[5], firstCollectionTimestamp)); // the source is done, but elements are still buffered. The source output watermark should be @@ -327,7 +327,9 @@ public void getWatermarkMultiIdenticalInput() { CommittedBundle createBundle = bundleFactory .createBundle(impulse) - .add(WindowedValues.timestampedValueInGlobalWindow(new byte[1], new Instant(33536))) + .add( + WindowedValues.timestampedValueInGlobalWindow( + new byte[1], Instant.ofEpochMilli(33536))) .commit(clock.now()); Map, Collection>> initialInputs = @@ -345,7 +347,7 @@ public void getWatermarkMultiIdenticalInput() { tstMgr.refreshAll(); TransformWatermarks flattenWms = tstMgr.getWatermarks(theFlatten); - assertThat(flattenWms.getInputWatermark(), equalTo(new Instant(33536))); + assertThat(flattenWms.getInputWatermark(), equalTo(Instant.ofEpochMilli(33536))); tstMgr.updateWatermarks( createBundle, @@ -356,7 +358,7 @@ public void getWatermarkMultiIdenticalInput() { BoundedWindow.TIMESTAMP_MAX_VALUE); tstMgr.refreshAll(); - assertThat(flattenWms.getInputWatermark(), equalTo(new Instant(33536))); + assertThat(flattenWms.getInputWatermark(), equalTo(Instant.ofEpochMilli(33536))); tstMgr.updateWatermarks( createBundle, @@ -378,16 +380,16 @@ public void getWatermarkForMultiConsumedCollection() { CommittedBundle impulseBundle = timestampedBundle( impulse, - TimestampedValue.of(new byte[1], new Instant(1_000_000L)), - TimestampedValue.of(new byte[2], new Instant(1234L)), - TimestampedValue.of(new byte[3], new Instant(-1000L))); + TimestampedValue.of(new byte[1], Instant.ofEpochMilli(1_000_000L)), + TimestampedValue.of(new byte[2], Instant.ofEpochMilli(1234L)), + TimestampedValue.of(new byte[3], Instant.ofEpochMilli(-1000L))); manager.updateWatermarks( null, TimerUpdate.empty(), graph.getProducer(impulse), null, Collections.>singleton(impulseBundle), - new Instant(Long.MAX_VALUE)); + Instant.ofEpochMilli(Long.MAX_VALUE)); manager.refreshAll(); TransformWatermarks impulseAfterProducing = manager.getWatermarks(graph.getProducer(impulse)); assertThat( @@ -397,9 +399,9 @@ public void getWatermarkForMultiConsumedCollection() { CommittedBundle> keyBundle = timestampedBundle( keyed, - TimestampedValue.of(KV.of("MyKey", new byte[1]), new Instant(1_000_000L)), - TimestampedValue.of(KV.of("MyKey", new byte[2]), new Instant(1234L)), - TimestampedValue.of(KV.of("MyKey", new byte[3]), new Instant(-1000L))); + TimestampedValue.of(KV.of("MyKey", new byte[1]), Instant.ofEpochMilli(1_000_000L)), + TimestampedValue.of(KV.of("MyKey", new byte[2]), Instant.ofEpochMilli(1234L)), + TimestampedValue.of(KV.of("MyKey", new byte[3]), Instant.ofEpochMilli(-1000L))); manager.updateWatermarks( impulseBundle, TimerUpdate.empty(), @@ -415,11 +417,13 @@ public void getWatermarkForMultiConsumedCollection() { keyedWatermarks.getOutputWatermark(), not(lessThan(BoundedWindow.TIMESTAMP_MAX_VALUE))); TransformWatermarks filteredWatermarks = manager.getWatermarks(graph.getProducer(filtered)); - assertThat(filteredWatermarks.getInputWatermark(), not(greaterThan(new Instant(-1000L)))); - assertThat(filteredWatermarks.getOutputWatermark(), not(greaterThan(new Instant(-1000L)))); + assertThat( + filteredWatermarks.getInputWatermark(), not(greaterThan(Instant.ofEpochMilli(-1000L)))); + assertThat( + filteredWatermarks.getOutputWatermark(), not(greaterThan(Instant.ofEpochMilli(-1000L)))); CommittedBundle filteredBundle = - timestampedBundle(filtered, TimestampedValue.of(new byte[2], new Instant(1234L))); + timestampedBundle(filtered, TimestampedValue.of(new byte[2], Instant.ofEpochMilli(1234L))); manager.updateWatermarks( impulseBundle, TimerUpdate.empty(), @@ -447,35 +451,35 @@ public void updateWatermarkWithWatermarkHolds() { CommittedBundle impulseBundle = timestampedBundle( impulse, - TimestampedValue.of(new byte[1], new Instant(1_000_000L)), - TimestampedValue.of(new byte[2], new Instant(1234L)), - TimestampedValue.of(new byte[3], new Instant(-1000L))); + TimestampedValue.of(new byte[1], Instant.ofEpochMilli(1_000_000L)), + TimestampedValue.of(new byte[2], Instant.ofEpochMilli(1234L)), + TimestampedValue.of(new byte[3], Instant.ofEpochMilli(-1000L))); manager.updateWatermarks( null, TimerUpdate.empty(), graph.getProducer(impulse), null, Collections.>singleton(impulseBundle), - new Instant(Long.MAX_VALUE)); + Instant.ofEpochMilli(Long.MAX_VALUE)); CommittedBundle> keyBundle = timestampedBundle( keyed, - TimestampedValue.of(KV.of("MyKey", new byte[1]), new Instant(1_000_000L)), - TimestampedValue.of(KV.of("MyKey", new byte[2]), new Instant(1234L)), - TimestampedValue.of(KV.of("MyKey", new byte[3]), new Instant(-1000L))); + TimestampedValue.of(KV.of("MyKey", new byte[1]), Instant.ofEpochMilli(1_000_000L)), + TimestampedValue.of(KV.of("MyKey", new byte[2]), Instant.ofEpochMilli(1234L)), + TimestampedValue.of(KV.of("MyKey", new byte[3]), Instant.ofEpochMilli(-1000L))); manager.updateWatermarks( impulseBundle, TimerUpdate.empty(), graph.getProducer(keyed), impulseBundle.withElements(Collections.emptyList()), Collections.>singleton(keyBundle), - new Instant(500L)); + Instant.ofEpochMilli(500L)); manager.refreshAll(); TransformWatermarks keyedWatermarks = manager.getWatermarks(graph.getProducer(keyed)); assertThat( keyedWatermarks.getInputWatermark(), not(lessThan(BoundedWindow.TIMESTAMP_MAX_VALUE))); - assertThat(keyedWatermarks.getOutputWatermark(), not(greaterThan(new Instant(500L)))); + assertThat(keyedWatermarks.getOutputWatermark(), not(greaterThan(Instant.ofEpochMilli(500L)))); } /** @@ -488,14 +492,19 @@ public void updateWatermarkWithKeyedWatermarkHolds() { bundleFactory .createKeyedBundle(StructuralKey.of("Odd", StringUtf8Coder.of()), impulse) .add( - WindowedValues.timestampedValueInGlobalWindow(new byte[1], new Instant(1_000_000L))) - .add(WindowedValues.timestampedValueInGlobalWindow(new byte[3], new Instant(-1000L))) + WindowedValues.timestampedValueInGlobalWindow( + new byte[1], Instant.ofEpochMilli(1_000_000L))) + .add( + WindowedValues.timestampedValueInGlobalWindow( + new byte[3], Instant.ofEpochMilli(-1000L))) .commit(clock.now()); CommittedBundle secondKeyBundle = bundleFactory .createKeyedBundle(StructuralKey.of("Even", StringUtf8Coder.of()), impulse) - .add(WindowedValues.timestampedValueInGlobalWindow(new byte[2], new Instant(1234L))) + .add( + WindowedValues.timestampedValueInGlobalWindow( + new byte[2], Instant.ofEpochMilli(1234L))) .commit(clock.now()); manager.updateWatermarks( @@ -512,20 +521,21 @@ public void updateWatermarkWithKeyedWatermarkHolds() { graph.getProducer(filtered), firstKeyBundle.withElements(Collections.emptyList()), Collections.emptyList(), - new Instant(-1000L)); + Instant.ofEpochMilli(-1000L)); manager.updateWatermarks( secondKeyBundle, TimerUpdate.empty(), graph.getProducer(filtered), secondKeyBundle.withElements(Collections.emptyList()), Collections.emptyList(), - new Instant(1234L)); + Instant.ofEpochMilli(1234L)); manager.refreshAll(); TransformWatermarks filteredWatermarks = manager.getWatermarks(graph.getProducer(filtered)); assertThat( filteredWatermarks.getInputWatermark(), not(lessThan(BoundedWindow.TIMESTAMP_MAX_VALUE))); - assertThat(filteredWatermarks.getOutputWatermark(), not(greaterThan(new Instant(-1000L)))); + assertThat( + filteredWatermarks.getOutputWatermark(), not(greaterThan(Instant.ofEpochMilli(-1000L)))); CommittedBundle fauxFirstKeyTimerBundle = bundleFactory @@ -540,7 +550,7 @@ public void updateWatermarkWithKeyedWatermarkHolds() { BoundedWindow.TIMESTAMP_MAX_VALUE); manager.refreshAll(); - assertThat(filteredWatermarks.getOutputWatermark(), equalTo(new Instant(1234L))); + assertThat(filteredWatermarks.getOutputWatermark(), equalTo(Instant.ofEpochMilli(1234L))); CommittedBundle fauxSecondKeyTimerBundle = bundleFactory @@ -552,9 +562,9 @@ public void updateWatermarkWithKeyedWatermarkHolds() { graph.getProducer(filtered), fauxSecondKeyTimerBundle.withElements(Collections.emptyList()), Collections.emptyList(), - new Instant(5678L)); + Instant.ofEpochMilli(5678L)); manager.refreshAll(); - assertThat(filteredWatermarks.getOutputWatermark(), equalTo(new Instant(5678L))); + assertThat(filteredWatermarks.getOutputWatermark(), equalTo(Instant.ofEpochMilli(5678L))); manager.updateWatermarks( fauxSecondKeyTimerBundle, @@ -582,10 +592,10 @@ public void updateOutputWatermarkShouldBeMonotonic() { graph.getProducer(impulse), null, Collections.>singleton(firstInput), - new Instant(0L)); + Instant.ofEpochMilli(0L)); manager.refreshAll(); TransformWatermarks firstWatermarks = manager.getWatermarks(graph.getProducer(impulse)); - assertThat(firstWatermarks.getOutputWatermark(), equalTo(new Instant(0L))); + assertThat(firstWatermarks.getOutputWatermark(), equalTo(Instant.ofEpochMilli(0L))); CommittedBundle secondInput = bundleFactory.createBundle(impulse).commit(BoundedWindow.TIMESTAMP_MAX_VALUE); @@ -595,10 +605,10 @@ public void updateOutputWatermarkShouldBeMonotonic() { graph.getProducer(impulse), null, Collections.>singleton(secondInput), - new Instant(-250L)); + Instant.ofEpochMilli(-250L)); manager.refreshAll(); TransformWatermarks secondWatermarks = manager.getWatermarks(graph.getProducer(impulse)); - assertThat(secondWatermarks.getOutputWatermark(), not(lessThan(new Instant(0L)))); + assertThat(secondWatermarks.getOutputWatermark(), not(lessThan(Instant.ofEpochMilli(0L)))); } /** @@ -610,35 +620,35 @@ public void updateWatermarkWithHoldsShouldBeMonotonic() { CommittedBundle impulseBundle = timestampedBundle( impulse, - TimestampedValue.of(new byte[1], new Instant(1_000_000L)), - TimestampedValue.of(new byte[2], new Instant(1234L)), - TimestampedValue.of(new byte[3], new Instant(-1000L))); + TimestampedValue.of(new byte[1], Instant.ofEpochMilli(1_000_000L)), + TimestampedValue.of(new byte[2], Instant.ofEpochMilli(1234L)), + TimestampedValue.of(new byte[3], Instant.ofEpochMilli(-1000L))); manager.updateWatermarks( null, TimerUpdate.empty(), graph.getProducer(impulse), null, Collections.>singleton(impulseBundle), - new Instant(Long.MAX_VALUE)); + Instant.ofEpochMilli(Long.MAX_VALUE)); CommittedBundle> keyBundle = timestampedBundle( keyed, - TimestampedValue.of(KV.of("MyKey", new byte[1]), new Instant(1_000_000L)), - TimestampedValue.of(KV.of("MyKey", new byte[2]), new Instant(1234L)), - TimestampedValue.of(KV.of("MyKey", new byte[3]), new Instant(-1000L))); + TimestampedValue.of(KV.of("MyKey", new byte[1]), Instant.ofEpochMilli(1_000_000L)), + TimestampedValue.of(KV.of("MyKey", new byte[2]), Instant.ofEpochMilli(1234L)), + TimestampedValue.of(KV.of("MyKey", new byte[3]), Instant.ofEpochMilli(-1000L))); manager.updateWatermarks( impulseBundle, TimerUpdate.empty(), graph.getProducer(keyed), impulseBundle.withElements(Collections.emptyList()), Collections.>singleton(keyBundle), - new Instant(500L)); + Instant.ofEpochMilli(500L)); manager.refreshAll(); TransformWatermarks keyedWatermarks = manager.getWatermarks(graph.getProducer(keyed)); assertThat( keyedWatermarks.getInputWatermark(), not(lessThan(BoundedWindow.TIMESTAMP_MAX_VALUE))); - assertThat(keyedWatermarks.getOutputWatermark(), not(greaterThan(new Instant(500L)))); + assertThat(keyedWatermarks.getOutputWatermark(), not(greaterThan(Instant.ofEpochMilli(500L)))); Instant oldOutputWatermark = keyedWatermarks.getOutputWatermark(); TransformWatermarks updatedWatermarks = manager.getWatermarks(graph.getProducer(keyed)); @@ -653,9 +663,9 @@ public void updateWatermarkWithHoldsShouldBeMonotonic() { public void updateWatermarkWithUnprocessedElements() { WindowedValue first = WindowedValues.valueInGlobalWindow(new byte[1]); WindowedValue second = - WindowedValues.timestampedValueInGlobalWindow(new byte[2], new Instant(-1000L)); + WindowedValues.timestampedValueInGlobalWindow(new byte[2], Instant.ofEpochMilli(-1000L)); WindowedValue third = - WindowedValues.timestampedValueInGlobalWindow(new byte[3], new Instant(1234L)); + WindowedValues.timestampedValueInGlobalWindow(new byte[3], Instant.ofEpochMilli(1234L)); CommittedBundle impulseBundle = bundleFactory.createBundle(impulse).add(first).add(second).add(third).commit(clock.now()); @@ -680,18 +690,18 @@ public void updateWatermarkWithUnprocessedElements() { BoundedWindow.TIMESTAMP_MAX_VALUE); TransformWatermarks keyedWatermarks = manager.getWatermarks(graph.getProducer(keyed)); // the unprocessed second and third are readded to pending - assertThat(keyedWatermarks.getInputWatermark(), not(greaterThan(new Instant(-1000L)))); + assertThat(keyedWatermarks.getInputWatermark(), not(greaterThan(Instant.ofEpochMilli(-1000L)))); } @Test public void updateWatermarkWithCompletedElementsNotPending() { WindowedValue first = - WindowedValues.timestampedValueInGlobalWindow(new byte[1], new Instant(22)); + WindowedValues.timestampedValueInGlobalWindow(new byte[1], Instant.ofEpochMilli(22)); CommittedBundle impulseBundle = bundleFactory.createBundle(impulse).add(first).commit(clock.now()); WindowedValue second = - WindowedValues.timestampedValueInGlobalWindow(new byte[2], new Instant(22)); + WindowedValues.timestampedValueInGlobalWindow(new byte[2], Instant.ofEpochMilli(22)); CommittedBundle neverImpulseBundle = bundleFactory.createBundle(impulse).add(second).commit(clock.now()); @@ -713,18 +723,18 @@ public void updateWatermarkWithCompletedElementsNotPending() { manager.refreshAll(); TransformWatermarks filteredWms = manager.getWatermarks(graph.getProducer(filtered)); - assertThat(filteredWms.getInputWatermark(), equalTo(new Instant(22L))); + assertThat(filteredWms.getInputWatermark(), equalTo(Instant.ofEpochMilli(22L))); } /** Demonstrates that updateWatermarks in the presence of late data is monotonic. */ @Test public void updateWatermarkWithLateData() { - Instant sourceWatermark = new Instant(1_000_000L); + Instant sourceWatermark = Instant.ofEpochMilli(1_000_000L); CommittedBundle impulseBundle = timestampedBundle( impulse, TimestampedValue.of(new byte[1], sourceWatermark), - TimestampedValue.of(new byte[2], new Instant(1234L))); + TimestampedValue.of(new byte[2], Instant.ofEpochMilli(1234L))); manager.updateWatermarks( null, @@ -738,7 +748,7 @@ public void updateWatermarkWithLateData() { timestampedBundle( keyed, TimestampedValue.of(KV.of("MyKey", new byte[1]), sourceWatermark), - TimestampedValue.of(KV.of("MyKey", new byte[2]), new Instant(1234L))); + TimestampedValue.of(KV.of("MyKey", new byte[2]), Instant.ofEpochMilli(1234L))); // Finish processing the on-time data. The watermarks should progress to be equal to the source manager.updateWatermarks( @@ -754,7 +764,7 @@ public void updateWatermarkWithLateData() { assertThat(onTimeWatermarks.getOutputWatermark(), equalTo(sourceWatermark)); CommittedBundle lateDataBundle = - timestampedBundle(impulse, TimestampedValue.of(new byte[3], new Instant(-1000L))); + timestampedBundle(impulse, TimestampedValue.of(new byte[3], Instant.ofEpochMilli(-1000L))); // the late data arrives in a downstream PCollection after its watermark has advanced past it; // we don't advance the watermark past the current watermark until we've consumed the late data manager.updateWatermarks( @@ -763,10 +773,10 @@ public void updateWatermarkWithLateData() { graph.getProducer(impulse), impulseBundle.withElements(Collections.emptyList()), Collections.>singleton(lateDataBundle), - new Instant(2_000_000L)); + Instant.ofEpochMilli(2_000_000L)); manager.refreshAll(); TransformWatermarks bufferedLateWm = manager.getWatermarks(graph.getProducer(impulse)); - assertThat(bufferedLateWm.getOutputWatermark(), equalTo(new Instant(2_000_000L))); + assertThat(bufferedLateWm.getOutputWatermark(), equalTo(Instant.ofEpochMilli(2_000_000L))); // The input watermark should be held to its previous value (not advanced due to late data; not // moved backwards in the presence of watermarks due to monotonicity). @@ -776,7 +786,7 @@ public void updateWatermarkWithLateData() { CommittedBundle> lateKeyedBundle = timestampedBundle( - keyed, TimestampedValue.of(KV.of("MyKey", new byte[3]), new Instant(-1000L))); + keyed, TimestampedValue.of(KV.of("MyKey", new byte[3]), Instant.ofEpochMilli(-1000L))); manager.updateWatermarks( lateDataBundle, TimerUpdate.empty(), @@ -864,7 +874,7 @@ public void getWatermarksAfterHoldAndEmptyOutput() { graph.getProducer(impulse), null, Collections.>singleton(firstImpulseOutput), - new Instant(12_000L)); + Instant.ofEpochMilli(12_000L)); CommittedBundle firstFilterOutput = multiWindowedBundle(filtered); manager.updateWatermarks( @@ -873,11 +883,14 @@ public void getWatermarksAfterHoldAndEmptyOutput() { graph.getProducer(filtered), firstImpulseOutput.withElements(Collections.emptyList()), Collections.>singleton(firstFilterOutput), - new Instant(10_000L)); + Instant.ofEpochMilli(10_000L)); manager.refreshAll(); TransformWatermarks firstFilterWatermarks = manager.getWatermarks(graph.getProducer(filtered)); - assertThat(firstFilterWatermarks.getInputWatermark(), not(lessThan(new Instant(12_000L)))); - assertThat(firstFilterWatermarks.getOutputWatermark(), not(greaterThan(new Instant(10_000L)))); + assertThat( + firstFilterWatermarks.getInputWatermark(), not(lessThan(Instant.ofEpochMilli(12_000L)))); + assertThat( + firstFilterWatermarks.getOutputWatermark(), + not(greaterThan(Instant.ofEpochMilli(10_000L)))); CommittedBundle emptyImpulseOutput = multiWindowedBundle(impulse); manager.updateWatermarks( @@ -900,7 +913,8 @@ public void getWatermarksAfterHoldAndEmptyOutput() { finishedFilterWatermarks.getInputWatermark(), not(lessThan(BoundedWindow.TIMESTAMP_MAX_VALUE))); assertThat( - finishedFilterWatermarks.getOutputWatermark(), not(greaterThan(new Instant(10_000L)))); + finishedFilterWatermarks.getOutputWatermark(), + not(greaterThan(Instant.ofEpochMilli(10_000L)))); } @Test @@ -921,7 +935,7 @@ public void getSynchronizedProcessingTimeInputWatermarksHeldToPendingBundles() { not(greaterThan(BoundedWindow.TIMESTAMP_MIN_VALUE))); CommittedBundle createOutput = - bundleFactory.createBundle(impulse).commit(new Instant(1250L)); + bundleFactory.createBundle(impulse).commit(Instant.ofEpochMilli(1250L)); manager.updateWatermarks( null, @@ -941,18 +955,18 @@ public void getSynchronizedProcessingTimeInputWatermarksHeldToPendingBundles() { assertThat( filterAfterProduced.getSynchronizedProcessingOutputTime(), not(greaterThan(clock.now()))); - clock.set(new Instant(1500L)); + clock.set(Instant.ofEpochMilli(1500L)); assertThat(createAfterUpdate.getSynchronizedProcessingInputTime(), equalTo(clock.now())); assertThat(createAfterUpdate.getSynchronizedProcessingOutputTime(), equalTo(clock.now())); assertThat( filterAfterProduced.getSynchronizedProcessingInputTime(), - not(greaterThan(new Instant(1250L)))); + not(greaterThan(Instant.ofEpochMilli(1250L)))); assertThat( filterAfterProduced.getSynchronizedProcessingOutputTime(), - not(greaterThan(new Instant(1250L)))); + not(greaterThan(Instant.ofEpochMilli(1250L)))); CommittedBundle filterOutputBundle = - bundleFactory.createBundle(impulseToFlatten).commit(new Instant(1250L)); + bundleFactory.createBundle(impulseToFlatten).commit(Instant.ofEpochMilli(1250L)); manager.updateWatermarks( createOutput, TimerUpdate.empty(), @@ -986,7 +1000,7 @@ public void getSynchronizedProcessingTimeOutputHeldToPendingTimers() { graph.getProducer(impulse), null, Collections.>singleton(impulseBundle), - new Instant(1248L)); + Instant.ofEpochMilli(1248L)); manager.refreshAll(); TransformWatermarks filteredWms = manager.getWatermarks(graph.getProducer(filtered)); @@ -1001,15 +1015,15 @@ public void getSynchronizedProcessingTimeOutputHeldToPendingTimers() { TimerData pastTimer = TimerData.of( StateNamespaces.global(), - new Instant(250L), - new Instant(250L), + Instant.ofEpochMilli(250L), + Instant.ofEpochMilli(250L), TimeDomain.PROCESSING_TIME, CausedByDrain.NORMAL); TimerData futureTimer = TimerData.of( StateNamespaces.global(), - new Instant(4096L), - new Instant(4096L), + Instant.ofEpochMilli(4096L), + Instant.ofEpochMilli(4096L), TimeDomain.PROCESSING_TIME, CausedByDrain.NORMAL); TimerUpdate timers = TimerUpdate.builder(key).setTimer(pastTimer).setTimer(futureTimer).build(); @@ -1072,11 +1086,11 @@ public void getSynchronizedProcessingTimeOutputHeldToPendingTimers() { BoundedWindow.TIMESTAMP_MAX_VALUE); manager.refreshAll(); - clock.set(new Instant(Long.MAX_VALUE)); + clock.set(Instant.ofEpochMilli(Long.MAX_VALUE)); assertThat( filteredDoubledWms.getSynchronizedProcessingOutputTime(), - not(greaterThan(new Instant(4096)))); + not(greaterThan(Instant.ofEpochMilli(4096)))); } /** @@ -1099,7 +1113,7 @@ public void getSynchronizedProcessingTimeOutputTimeIsMonotonic() { not(greaterThan(BoundedWindow.TIMESTAMP_MIN_VALUE))); CommittedBundle createOutput = - bundleFactory.createBundle(impulse).commit(new Instant(1250L)); + bundleFactory.createBundle(impulse).commit(Instant.ofEpochMilli(1250L)); manager.updateWatermarks( null, @@ -1116,7 +1130,7 @@ public void getSynchronizedProcessingTimeOutputTimeIsMonotonic() { createAfterUpdate.getSynchronizedProcessingOutputTime(), not(greaterThan(clock.now()))); CommittedBundle createSecondOutput = - bundleFactory.createBundle(impulse).commit(new Instant(750L)); + bundleFactory.createBundle(impulse).commit(Instant.ofEpochMilli(750L)); manager.updateWatermarks( null, TimerUpdate.empty(), @@ -1139,12 +1153,12 @@ public void synchronizedProcessingInputTimeIsHeldToUpstreamProcessingTimeTimers( graph.getProducer(impulse), null, Collections.>singleton(impulseBundle), - new Instant(40_900L)); + Instant.ofEpochMilli(40_900L)); manager.refreshAll(); CommittedBundle filteredBundle = multiWindowedBundle(filtered, new byte[2], new byte[4]); - Instant upstreamHold = new Instant(2048L); + Instant upstreamHold = Instant.ofEpochMilli(2048L); TimerData upstreamProcessingTimer = TimerData.of( StateNamespaces.global(), @@ -1200,9 +1214,9 @@ public void synchronizedProcessingInputTimeIsHeldToPendingBundleTimes() { graph.getProducer(impulse), null, Collections.>singleton(impulseBundle), - new Instant(29_919_235L)); + Instant.ofEpochMilli(29_919_235L)); - Instant upstreamHold = new Instant(2048L); + Instant upstreamHold = Instant.ofEpochMilli(2048L); CommittedBundle filteredBundle = bundleFactory .createKeyedBundle(StructuralKey.of("key", StringUtf8Coder.of()), filtered) @@ -1238,28 +1252,28 @@ public void extractFiredTimersReturnsFiredEventTimeTimers() { graph.getProducer(impulse), null, Collections.singleton(impulseBundle), - new Instant(1500L)); + Instant.ofEpochMilli(1500L)); manager.refreshAll(); TimerData earliestTimer = TimerData.of( StateNamespaces.global(), - new Instant(1000), - new Instant(1000), + Instant.ofEpochMilli(1000), + Instant.ofEpochMilli(1000), TimeDomain.EVENT_TIME, CausedByDrain.NORMAL); TimerData middleTimer = TimerData.of( StateNamespaces.global(), - new Instant(5000L), - new Instant(5000L), + Instant.ofEpochMilli(5000L), + Instant.ofEpochMilli(5000L), TimeDomain.EVENT_TIME, CausedByDrain.NORMAL); TimerData lastTimer = TimerData.of( StateNamespaces.global(), - new Instant(10000L), - new Instant(10000L), + Instant.ofEpochMilli(10000L), + Instant.ofEpochMilli(10000L), TimeDomain.EVENT_TIME, CausedByDrain.NORMAL); StructuralKey key = StructuralKey.of(new byte[] {1, 4, 9}, ByteArrayCoder.of()); @@ -1276,7 +1290,7 @@ public void extractFiredTimersReturnsFiredEventTimeTimers() { graph.getProducer(filtered), impulseBundle.withElements(Collections.emptyList()), Collections.>singleton(multiWindowedBundle(impulseToFlatten)), - new Instant(1000L)); + Instant.ofEpochMilli(1000L)); manager.refreshAll(); Collection>> firstFiredTimers = @@ -1291,7 +1305,7 @@ public void extractFiredTimersReturnsFiredEventTimeTimers() { graph.getProducer(impulse), null, Collections.emptyList(), - new Instant(50_000L)); + Instant.ofEpochMilli(50_000L)); manager.refreshAll(); assertTrue(manager.extractFiredTimers().isEmpty()); @@ -1302,7 +1316,7 @@ public void extractFiredTimersReturnsFiredEventTimeTimers() { graph.getProducer(filtered), null, Collections.emptyList(), - new Instant(1000L)); + Instant.ofEpochMilli(1000L)); manager.refreshAll(); @@ -1330,27 +1344,27 @@ public void extractFiredTimersReturnsFiredProcessingTimeTimers() { graph.getProducer(impulse), null, Collections.singleton(impulseBundle), - new Instant(1500L)); + Instant.ofEpochMilli(1500L)); TimerData earliestTimer = TimerData.of( StateNamespaces.global(), - new Instant(999L), - new Instant(999L), + Instant.ofEpochMilli(999L), + Instant.ofEpochMilli(999L), TimeDomain.PROCESSING_TIME, CausedByDrain.NORMAL); TimerData middleTimer = TimerData.of( StateNamespaces.global(), - new Instant(5000L), - new Instant(5000L), + Instant.ofEpochMilli(5000L), + Instant.ofEpochMilli(5000L), TimeDomain.PROCESSING_TIME, CausedByDrain.NORMAL); TimerData lastTimer = TimerData.of( StateNamespaces.global(), - new Instant(10000L), - new Instant(10000L), + Instant.ofEpochMilli(10000L), + Instant.ofEpochMilli(10000L), TimeDomain.PROCESSING_TIME, CausedByDrain.NORMAL); StructuralKey key = StructuralKey.of(-12L, VarLongCoder.of()); @@ -1367,7 +1381,7 @@ public void extractFiredTimersReturnsFiredProcessingTimeTimers() { graph.getProducer(filtered), impulseBundle.withElements(Collections.emptyList()), Collections.>singleton(multiWindowedBundle(impulseToFlatten)), - new Instant(1000L)); + Instant.ofEpochMilli(1000L)); manager.refreshAll(); Collection>> firstFiredTimers = @@ -1376,14 +1390,14 @@ public void extractFiredTimersReturnsFiredProcessingTimeTimers() { FiredTimers> firstFired = Iterables.getOnlyElement(firstFiredTimers); assertThat(firstFired.getTimers(), contains(earliestTimer)); - clock.set(new Instant(50_000L)); + clock.set(Instant.ofEpochMilli(50_000L)); manager.updateWatermarks( null, TimerUpdate.empty(), graph.getProducer(impulse), null, Collections.emptyList(), - new Instant(50_000L)); + Instant.ofEpochMilli(50_000L)); manager.refreshAll(); assertTrue(manager.extractFiredTimers().isEmpty()); @@ -1393,7 +1407,7 @@ public void extractFiredTimersReturnsFiredProcessingTimeTimers() { graph.getProducer(filtered), null, Collections.emptyList(), - new Instant(1000L)); + Instant.ofEpochMilli(1000L)); manager.refreshAll(); @@ -1421,27 +1435,27 @@ public void extractFiredTimersReturnsFiredSynchronizedProcessingTimeTimers() { graph.getProducer(impulse), null, Collections.singleton(impulseBundle), - new Instant(1500L)); + Instant.ofEpochMilli(1500L)); TimerData earliestTimer = TimerData.of( StateNamespaces.global(), - new Instant(999L), - new Instant(999L), + Instant.ofEpochMilli(999L), + Instant.ofEpochMilli(999L), TimeDomain.SYNCHRONIZED_PROCESSING_TIME, CausedByDrain.NORMAL); TimerData middleTimer = TimerData.of( StateNamespaces.global(), - new Instant(5000L), - new Instant(5000L), + Instant.ofEpochMilli(5000L), + Instant.ofEpochMilli(5000L), TimeDomain.SYNCHRONIZED_PROCESSING_TIME, CausedByDrain.NORMAL); TimerData lastTimer = TimerData.of( StateNamespaces.global(), - new Instant(10000L), - new Instant(10000L), + Instant.ofEpochMilli(10000L), + Instant.ofEpochMilli(10000L), TimeDomain.SYNCHRONIZED_PROCESSING_TIME, CausedByDrain.NORMAL); StructuralKey key = StructuralKey.of(new byte[] {2, -2, 22}, ByteArrayCoder.of()); @@ -1458,7 +1472,7 @@ public void extractFiredTimersReturnsFiredSynchronizedProcessingTimeTimers() { graph.getProducer(filtered), impulseBundle.withElements(Collections.emptyList()), Collections.>singleton(multiWindowedBundle(impulseToFlatten)), - new Instant(1000L)); + Instant.ofEpochMilli(1000L)); manager.refreshAll(); Collection>> firstFiredTimers = @@ -1467,14 +1481,14 @@ public void extractFiredTimersReturnsFiredSynchronizedProcessingTimeTimers() { FiredTimers> firstFired = Iterables.getOnlyElement(firstFiredTimers); assertThat(firstFired.getTimers(), contains(earliestTimer)); - clock.set(new Instant(50_000L)); + clock.set(Instant.ofEpochMilli(50_000L)); manager.updateWatermarks( null, TimerUpdate.empty(), graph.getProducer(impulse), null, Collections.emptyList(), - new Instant(50_000L)); + Instant.ofEpochMilli(50_000L)); manager.refreshAll(); assertTrue(manager.extractFiredTimers().isEmpty()); @@ -1484,7 +1498,7 @@ public void extractFiredTimersReturnsFiredSynchronizedProcessingTimeTimers() { graph.getProducer(filtered), null, Collections.emptyList(), - new Instant(1000L)); + Instant.ofEpochMilli(1000L)); Collection>> secondFiredTimers = manager.extractFiredTimers(); @@ -1508,8 +1522,8 @@ public void processingTimeTimersCanBeReset() { TimerData.of( timerId, StateNamespaces.global(), - new Instant(5000L), - new Instant(5000L), + Instant.ofEpochMilli(5000L), + Instant.ofEpochMilli(5000L), TimeDomain.PROCESSING_TIME, CausedByDrain.NORMAL); @@ -1517,8 +1531,8 @@ public void processingTimeTimersCanBeReset() { TimerData.of( timerId, StateNamespaces.global(), - new Instant(10000L), - new Instant(10000L), + Instant.ofEpochMilli(10000L), + Instant.ofEpochMilli(10000L), TimeDomain.PROCESSING_TIME, CausedByDrain.NORMAL); @@ -1531,7 +1545,7 @@ public void processingTimeTimersCanBeReset() { graph.getProducer(impulse), null, Collections.emptyList(), - new Instant(5000L)); + Instant.ofEpochMilli(5000L)); manager.refreshAll(); // This update should override the previous timer. @@ -1541,10 +1555,10 @@ public void processingTimeTimersCanBeReset() { graph.getProducer(impulse), null, Collections.emptyList(), - new Instant(10000L)); + Instant.ofEpochMilli(10000L)); // Set clock past the timers. - clock.set(new Instant(50000L)); + clock.set(Instant.ofEpochMilli(50000L)); manager.refreshAll(); Collection>> firedTimers = manager.extractFiredTimers(); @@ -1566,16 +1580,16 @@ public void eventTimeTimersCanBeReset() { TimerData.of( timerId, StateNamespaces.global(), - new Instant(1000L), - new Instant(1000L), + Instant.ofEpochMilli(1000L), + Instant.ofEpochMilli(1000L), TimeDomain.EVENT_TIME, CausedByDrain.NORMAL); TimerData overridingTimer = TimerData.of( timerId, StateNamespaces.global(), - new Instant(2000L), - new Instant(2000L), + Instant.ofEpochMilli(2000L), + Instant.ofEpochMilli(2000L), TimeDomain.EVENT_TIME, CausedByDrain.NORMAL); @@ -1589,7 +1603,7 @@ public void eventTimeTimersCanBeReset() { graph.getProducer(filtered), impulseBundle.withElements(Collections.emptyList()), Collections.>singleton(multiWindowedBundle(impulseToFlatten)), - new Instant(1000L)); + Instant.ofEpochMilli(1000L)); manager.refreshAll(); // This update should override the previous timer. @@ -1599,7 +1613,7 @@ public void eventTimeTimersCanBeReset() { graph.getProducer(filtered), impulseBundle.withElements(Collections.emptyList()), Collections.>singleton(multiWindowedBundle(impulseToFlatten)), - new Instant(1000L)); + Instant.ofEpochMilli(1000L)); manager.refreshAll(); // Set WM past the timers. @@ -1609,7 +1623,7 @@ public void eventTimeTimersCanBeReset() { graph.getProducer(impulse), null, Collections.singleton(impulseBundle), - new Instant(3000L)); + Instant.ofEpochMilli(3000L)); manager.refreshAll(); Collection>> firstFiredTimers = @@ -1628,9 +1642,9 @@ public void inputWatermarkDuplicates() { "underTest", ImmutableList.of(mockWatermark), update -> {}); // Refresh - when(mockWatermark.get()).thenReturn(new Instant(0)); + when(mockWatermark.get()).thenReturn(Instant.ofEpochMilli(0)); underTest.refresh(); - assertEquals(new Instant(0), underTest.get()); + assertEquals(Instant.ofEpochMilli(0), underTest.get()); // Apply a timer update StructuralKey key = StructuralKey.of("key", StringUtf8Coder.of()); @@ -1638,16 +1652,16 @@ public void inputWatermarkDuplicates() { TimerData.of( "a", StateNamespaces.global(), - new Instant(100), - new Instant(100), + Instant.ofEpochMilli(100), + Instant.ofEpochMilli(100), TimeDomain.EVENT_TIME, CausedByDrain.NORMAL); TimerData timer2 = TimerData.of( "a", StateNamespaces.global(), - new Instant(200), - new Instant(200), + Instant.ofEpochMilli(200), + Instant.ofEpochMilli(200), TimeDomain.EVENT_TIME, CausedByDrain.NORMAL); underTest.updateTimers(TimerUpdate.builder(key).setTimer(timer1).setTimer(timer2).build()); @@ -1656,9 +1670,10 @@ public void inputWatermarkDuplicates() { assertEquals(timer2.getTimestamp(), underTest.getEarliestTimerTimestamp()); // Advance the input watermark - when(mockWatermark.get()).thenReturn(new Instant(1000)); + when(mockWatermark.get()).thenReturn(Instant.ofEpochMilli(1000)); underTest.refresh(); - assertEquals(new Instant(1000), underTest.get()); // input watermark is not held by timers + assertEquals( + Instant.ofEpochMilli(1000), underTest.get()); // input watermark is not held by timers // Examine the fired event time timers Map, List> fired = underTest.extractFiredEventTimeTimers(); @@ -1682,29 +1697,29 @@ public void timerUpdateBuilderBuildAddsAllAddedTimers() { TimerData set = TimerData.of( StateNamespaces.global(), - new Instant(10L), - new Instant(10L), + Instant.ofEpochMilli(10L), + Instant.ofEpochMilli(10L), TimeDomain.EVENT_TIME, CausedByDrain.NORMAL); TimerData deleted = TimerData.of( StateNamespaces.global(), - new Instant(24L), - new Instant(24L), + Instant.ofEpochMilli(24L), + Instant.ofEpochMilli(24L), TimeDomain.PROCESSING_TIME, CausedByDrain.NORMAL); TimerData completedOne = TimerData.of( StateNamespaces.global(), - new Instant(1024L), - new Instant(1024L), + Instant.ofEpochMilli(1024L), + Instant.ofEpochMilli(1024L), TimeDomain.SYNCHRONIZED_PROCESSING_TIME, CausedByDrain.NORMAL); TimerData completedTwo = TimerData.of( StateNamespaces.global(), - new Instant(2048L), - new Instant(2048L), + Instant.ofEpochMilli(2048L), + Instant.ofEpochMilli(2048L), TimeDomain.EVENT_TIME, CausedByDrain.NORMAL); @@ -1859,7 +1874,7 @@ private final CommittedBundle multiWindowedBundle(PCollection pc, T... Collection windows = ImmutableList.of( GlobalWindow.INSTANCE, - new IntervalWindow(BoundedWindow.TIMESTAMP_MIN_VALUE, new Instant(0))); + new IntervalWindow(BoundedWindow.TIMESTAMP_MIN_VALUE, Instant.ofEpochMilli(0))); for (T value : values) { bundle.add( WindowedValues.of(value, BoundedWindow.TIMESTAMP_MIN_VALUE, windows, PaneInfo.NO_FIRING)); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java index 7bb5d7dd1ebe..8a280988d540 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java @@ -61,7 +61,7 @@ /** Tests for {@link WindowEvaluatorFactory}. */ @RunWith(JUnit4.class) public class WindowEvaluatorFactoryTest { - private static final Instant EPOCH = new Instant(0); + private static final Instant EPOCH = Instant.ofEpochMilli(0); private PCollection input; private WindowEvaluatorFactory factory; @@ -71,12 +71,15 @@ public class WindowEvaluatorFactoryTest { private BundleFactory bundleFactory; private WindowedValue valueInGlobalWindow = - WindowedValues.timestampedValueInGlobalWindow(3L, new Instant(2L)); + WindowedValues.timestampedValueInGlobalWindow(3L, Instant.ofEpochMilli(2L)); private final PaneInfo intervalWindowPane = PaneInfo.createPane(false, false, Timing.LATE, 3, 2); private WindowedValue valueInIntervalWindow = WindowedValues.of( - 2L, new Instant(-10L), new IntervalWindow(new Instant(-100), EPOCH), intervalWindowPane); + 2L, + Instant.ofEpochMilli(-10L), + new IntervalWindow(Instant.ofEpochMilli(-100), EPOCH), + intervalWindowPane); private IntervalWindow intervalWindow1 = new IntervalWindow(EPOCH, BoundedWindow.TIMESTAMP_MAX_VALUE); @@ -126,10 +129,10 @@ public void singleWindowFnSucceeds() throws Exception { committed.getElements(), containsInAnyOrder( // value in global window - isSingleWindowedValue(3L, new Instant(2L), firstSecondWindow, NO_FIRING), + isSingleWindowedValue(3L, Instant.ofEpochMilli(2L), firstSecondWindow, NO_FIRING), // value in just interval window - isSingleWindowedValue(2L, new Instant(-10L), thirdWindow, intervalWindowPane), + isSingleWindowedValue(2L, Instant.ofEpochMilli(-10L), thirdWindow, intervalWindowPane), // value in global window and two interval windows isSingleWindowedValue( diff --git a/runners/extensions-java/metrics/src/test/java/org/apache/beam/runners/extensions/metrics/CustomMetricQueryResults.java b/runners/extensions-java/metrics/src/test/java/org/apache/beam/runners/extensions/metrics/CustomMetricQueryResults.java index 438bf28f3eb8..99b60edf53c6 100644 --- a/runners/extensions-java/metrics/src/test/java/org/apache/beam/runners/extensions/metrics/CustomMetricQueryResults.java +++ b/runners/extensions-java/metrics/src/test/java/org/apache/beam/runners/extensions/metrics/CustomMetricQueryResults.java @@ -73,8 +73,8 @@ public List> getGauges() { return makeResults( "s3", "n3", - GaugeResult.create(100L, new Instant(345862800L)), - GaugeResult.create(120L, new Instant(345862800L))); + GaugeResult.create(100L, Instant.ofEpochMilli(345862800L)), + GaugeResult.create(120L, Instant.ofEpochMilli(345862800L))); } @Override diff --git a/runners/flink/1.20/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java b/runners/flink/1.20/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java index dca2f3075aa0..665169fcd72a 100644 --- a/runners/flink/1.20/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java +++ b/runners/flink/1.20/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java @@ -1701,12 +1701,12 @@ void deleteTimerInternal(TimerData timer) { @Override public Instant currentProcessingTime() { - return new Instant(timerService.currentProcessingTime()); + return Instant.ofEpochMilli(timerService.currentProcessingTime()); } @Override public @Nullable Instant currentSynchronizedProcessingTime() { - return new Instant(timerService.currentProcessingTime()); + return Instant.ofEpochMilli(timerService.currentProcessingTime()); } @Override @@ -1731,16 +1731,16 @@ public Instant currentInputWatermarkTime() { // BoundedWindow.TIMESTAMP_MAX_VALUE (9223372036854775)]. To ensure the timestamp visible to // the users follow the Beam convention, we just use the Beam range instead. return timerService.currentWatermark() == Long.MAX_VALUE - ? new Instant(Long.MAX_VALUE) + ? Instant.ofEpochMilli(Long.MAX_VALUE) : BoundedWindow.TIMESTAMP_MIN_VALUE; } else { - return new Instant(getEffectiveInputWatermark()); + return Instant.ofEpochMilli(getEffectiveInputWatermark()); } } @Override public @Nullable Instant currentOutputWatermarkTime() { - return new Instant(currentOutputWatermark); + return Instant.ofEpochMilli(currentOutputWatermark); } /** diff --git a/runners/flink/2.0/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java b/runners/flink/2.0/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java index f83e719ed0b9..df84b47b0c94 100644 --- a/runners/flink/2.0/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java +++ b/runners/flink/2.0/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java @@ -1701,12 +1701,12 @@ void deleteTimerInternal(TimerData timer) { @Override public Instant currentProcessingTime() { - return new Instant(timerService.currentProcessingTime()); + return Instant.ofEpochMilli(timerService.currentProcessingTime()); } @Override public @Nullable Instant currentSynchronizedProcessingTime() { - return new Instant(timerService.currentProcessingTime()); + return Instant.ofEpochMilli(timerService.currentProcessingTime()); } @Override @@ -1731,16 +1731,16 @@ public Instant currentInputWatermarkTime() { // BoundedWindow.TIMESTAMP_MAX_VALUE (9223372036854775)]. To ensure the timestamp visible to // the users follow the Beam convention, we just use the Beam range instead. return timerService.currentWatermark() == Long.MAX_VALUE - ? new Instant(Long.MAX_VALUE) + ? Instant.ofEpochMilli(Long.MAX_VALUE) : BoundedWindow.TIMESTAMP_MIN_VALUE; } else { - return new Instant(getEffectiveInputWatermark()); + return Instant.ofEpochMilli(getEffectiveInputWatermark()); } } @Override public @Nullable Instant currentOutputWatermarkTime() { - return new Instant(currentOutputWatermark); + return Instant.ofEpochMilli(currentOutputWatermark); } /** diff --git a/runners/flink/2.0/src/test/java/org/apache/beam/runners/flink/FlinkSavepointTest.java b/runners/flink/2.0/src/test/java/org/apache/beam/runners/flink/FlinkSavepointTest.java index bcca529a64b9..6dff9751a62a 100644 --- a/runners/flink/2.0/src/test/java/org/apache/beam/runners/flink/FlinkSavepointTest.java +++ b/runners/flink/2.0/src/test/java/org/apache/beam/runners/flink/FlinkSavepointTest.java @@ -345,7 +345,7 @@ public KV apply(byte[] input) { @ProcessElement public void processElement( ProcessContext context, @TimerId("timer") Timer timer) { - timer.set(new Instant(0)); + timer.set(Instant.ofEpochMilli(0)); } @OnTimer("timer") @@ -359,7 +359,9 @@ public void onTimer( LOG.debug("triggering timer {}", current); nextInteger.write(current + 1); // Trigger timer again and continue to hold back the watermark - timer.withOutputTimestamp(new Instant(0)).set(context.fireTimestamp()); + timer + .withOutputTimestamp(Instant.ofEpochMilli(0)) + .set(context.fireTimestamp()); } })); } else { diff --git a/runners/flink/2.0/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java b/runners/flink/2.0/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java index 6d74c51d7d9b..846cf8e7c858 100644 --- a/runners/flink/2.0/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java +++ b/runners/flink/2.0/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java @@ -92,21 +92,22 @@ public void testWatermarkHoldsPersistence() throws Exception { WatermarkHoldState fixedWindow = stateInternals.state( StateNamespaces.window( - IntervalWindow.getCoder(), new IntervalWindow(new Instant(0), new Instant(10))), + IntervalWindow.getCoder(), + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10))), stateTag); - Instant noHold = new Instant(Long.MAX_VALUE); + Instant noHold = Instant.ofEpochMilli(Long.MAX_VALUE); assertThat(stateInternals.minWatermarkHoldMs(), is(noHold.getMillis())); - Instant high = new Instant(10); + Instant high = Instant.ofEpochMilli(10); globalWindow.add(high); assertThat(stateInternals.minWatermarkHoldMs(), is(high.getMillis())); - Instant middle = new Instant(5); + Instant middle = Instant.ofEpochMilli(5); fixedWindow.add(middle); assertThat(stateInternals.minWatermarkHoldMs(), is(middle.getMillis())); - Instant low = new Instant(1); + Instant low = Instant.ofEpochMilli(1); globalWindow.add(low); assertThat(stateInternals.minWatermarkHoldMs(), is(low.getMillis())); @@ -145,7 +146,8 @@ public void testWatermarkHoldsPersistence() throws Exception { fixedWindow = stateInternals.state( StateNamespaces.window( - IntervalWindow.getCoder(), new IntervalWindow(new Instant(0), new Instant(10))), + IntervalWindow.getCoder(), + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10))), stateTag); // Watermark hold across all keys should be unchanged diff --git a/runners/flink/2.0/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapperTest.java b/runners/flink/2.0/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapperTest.java index 48939b0cbbf1..da1f67a4f3d8 100644 --- a/runners/flink/2.0/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapperTest.java +++ b/runners/flink/2.0/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapperTest.java @@ -974,7 +974,7 @@ public Instant getWatermark() { if (currentIdx >= data.size() - 1) { return BoundedWindow.TIMESTAMP_MAX_VALUE; } - return new Instant(currentIdx); + return Instant.ofEpochMilli(currentIdx); } @Override @@ -998,7 +998,7 @@ public T getCurrent() throws NoSuchElementException { @Override public Instant getCurrentTimestamp() throws NoSuchElementException { if (currentIdx >= 0 && currentIdx < data.size()) { - return new Instant(currentIdx); + return Instant.ofEpochMilli(currentIdx); } throw new NoSuchElementException(); } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java index 2bf0d40cd5f2..e555d80b897d 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java @@ -1704,12 +1704,12 @@ void deleteTimerInternal(TimerData timer) { @Override public Instant currentProcessingTime() { - return new Instant(timerService.currentProcessingTime()); + return Instant.ofEpochMilli(timerService.currentProcessingTime()); } @Override public @Nullable Instant currentSynchronizedProcessingTime() { - return new Instant(timerService.currentProcessingTime()); + return Instant.ofEpochMilli(timerService.currentProcessingTime()); } @Override @@ -1734,16 +1734,16 @@ public Instant currentInputWatermarkTime() { // BoundedWindow.TIMESTAMP_MAX_VALUE (9223372036854775)]. To ensure the timestamp visible to // the users follow the Beam convention, we just use the Beam range instead. return timerService.currentWatermark() == Long.MAX_VALUE - ? new Instant(Long.MAX_VALUE) + ? Instant.ofEpochMilli(Long.MAX_VALUE) : BoundedWindow.TIMESTAMP_MIN_VALUE; } else { - return new Instant(getEffectiveInputWatermark()); + return Instant.ofEpochMilli(getEffectiveInputWatermark()); } } @Override public @Nullable Instant currentOutputWatermarkTime() { - return new Instant(currentOutputWatermark); + return Instant.ofEpochMilli(currentOutputWatermark); } /** diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkSavepointTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkSavepointTest.java index c7dfe7f6cb78..2ff44b2f0500 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkSavepointTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkSavepointTest.java @@ -344,7 +344,7 @@ public KV apply(byte[] input) { @ProcessElement public void processElement( ProcessContext context, @TimerId("timer") Timer timer) { - timer.set(new Instant(0)); + timer.set(Instant.ofEpochMilli(0)); } @OnTimer("timer") @@ -358,7 +358,9 @@ public void onTimer( LOG.debug("triggering timer {}", current); nextInteger.write(current + 1); // Trigger timer again and continue to hold back the watermark - timer.withOutputTimestamp(new Instant(0)).set(context.fireTimestamp()); + timer + .withOutputTimestamp(Instant.ofEpochMilli(0)) + .set(context.fireTimestamp()); } })); } else { diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java index 2324a262acc0..f230110a03d0 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java @@ -92,21 +92,22 @@ public void testWatermarkHoldsPersistence() throws Exception { WatermarkHoldState fixedWindow = stateInternals.state( StateNamespaces.window( - IntervalWindow.getCoder(), new IntervalWindow(new Instant(0), new Instant(10))), + IntervalWindow.getCoder(), + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10))), stateTag); - Instant noHold = new Instant(Long.MAX_VALUE); + Instant noHold = Instant.ofEpochMilli(Long.MAX_VALUE); assertThat(stateInternals.minWatermarkHoldMs(), is(noHold.getMillis())); - Instant high = new Instant(10); + Instant high = Instant.ofEpochMilli(10); globalWindow.add(high); assertThat(stateInternals.minWatermarkHoldMs(), is(high.getMillis())); - Instant middle = new Instant(5); + Instant middle = Instant.ofEpochMilli(5); fixedWindow.add(middle); assertThat(stateInternals.minWatermarkHoldMs(), is(middle.getMillis())); - Instant low = new Instant(1); + Instant low = Instant.ofEpochMilli(1); globalWindow.add(low); assertThat(stateInternals.minWatermarkHoldMs(), is(low.getMillis())); @@ -145,7 +146,8 @@ public void testWatermarkHoldsPersistence() throws Exception { fixedWindow = stateInternals.state( StateNamespaces.window( - IntervalWindow.getCoder(), new IntervalWindow(new Instant(0), new Instant(10))), + IntervalWindow.getCoder(), + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10))), stateTag); // Watermark hold across all keys should be unchanged diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java index 7650df3072b2..db799d33a959 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java @@ -81,7 +81,7 @@ public void processElement(ProcessContext c) { String userName = record.getValue(); if (userName != null) { // Sets the implicit timestamp field to be used in windowing. - c.outputWithTimestamp(userName, new Instant(timestamp)); + c.outputWithTimestamp(userName, Instant.ofEpochMilli(timestamp)); } } } diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupByWithNullValuesTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupByWithNullValuesTest.java index 1797a0e40ac1..c1841fafbac1 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupByWithNullValuesTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupByWithNullValuesTest.java @@ -57,7 +57,7 @@ public void testGroupByWithNullValues() { new SerializableFunction() { @Override public Instant apply(Long input) { - return new Instant(input); + return Instant.ofEpochMilli(input); } })) .apply(Window.into(FixedWindows.of(Duration.millis(10)))) diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsTest.java index 0625576a1b26..63a5077e1d32 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsTest.java @@ -158,7 +158,7 @@ public void processElement(ProcessContext c) throws Exception { String userName = (String) row.get("contributor_username"); if (userName != null) { // Sets the timestamp field to be used in windowing. - c.outputWithTimestamp(userName, new Instant(timestamp * 1000L)); + c.outputWithTimestamp(userName, Instant.ofEpochMilli(timestamp * 1000L)); } } })) diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java index 5c4975ffab01..aa9164518df6 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java @@ -277,7 +277,7 @@ public void testMultiOutputOutput() throws Exception { @Test public void testWatermarkContract() throws Exception { - final Instant timerTimestamp = new Instant(1000); + final Instant timerTimestamp = Instant.ofEpochMilli(1000); final Instant timerOutputTimestamp = timerTimestamp.minus(Duration.millis(1)); final String eventTimeMessage = "Event timer fired: "; final String processingTimeMessage = "Processing timer fired"; @@ -311,7 +311,7 @@ public void processElement( .withOutputTimestamp(timerOutputTimestamp) .set(timerTimestamp); processingTimer - .withOutputTimestamp(new Instant(10)) + .withOutputTimestamp(Instant.ofEpochMilli(10)) .offset(Duration.millis(timerTimestamp.getMillis())) .setRelative(); } @@ -389,11 +389,12 @@ public void onProcessingTime(OnTimerContext context) { testHarness.processWatermark(499); testHarness.setProcessingTime(0); - IntervalWindow window1 = new IntervalWindow(new Instant(0), Duration.millis(10_000)); + IntervalWindow window1 = new IntervalWindow(Instant.ofEpochMilli(0), Duration.millis(10_000)); // this should register the two timers above testHarness.processElement( - new StreamRecord<>(WindowedValues.of(13, new Instant(0), window1, PaneInfo.NO_FIRING))); + new StreamRecord<>( + WindowedValues.of(13, Instant.ofEpochMilli(0), window1, PaneInfo.NO_FIRING))); assertThat(stripStreamRecordFromWindowedValue(testHarness.getOutput()), emptyIterable()); @@ -425,7 +426,7 @@ public void onProcessingTime(OnTimerContext context) { window1, PaneInfo.NO_FIRING), WindowedValues.of( - processingTimeMessage, new Instant(10), window1, PaneInfo.NO_FIRING))); + processingTimeMessage, Instant.ofEpochMilli(10), window1, PaneInfo.NO_FIRING))); testHarness.close(); } @@ -644,15 +645,16 @@ public void processElement(ProcessContext context) { testHarness.processWatermark(0); - IntervalWindow window1 = new IntervalWindow(new Instant(0), Duration.millis(10)); + IntervalWindow window1 = new IntervalWindow(Instant.ofEpochMilli(0), Duration.millis(10)); // this should not be late testHarness.processElement( - new StreamRecord<>(WindowedValues.of(13, new Instant(0), window1, PaneInfo.NO_FIRING))); + new StreamRecord<>( + WindowedValues.of(13, Instant.ofEpochMilli(0), window1, PaneInfo.NO_FIRING))); assertThat( stripStreamRecordFromWindowedValue(testHarness.getOutput()), - contains(WindowedValues.of("13", new Instant(0), window1, PaneInfo.NO_FIRING))); + contains(WindowedValues.of("13", Instant.ofEpochMilli(0), window1, PaneInfo.NO_FIRING))); testHarness.getOutput().clear(); @@ -660,11 +662,12 @@ public void processElement(ProcessContext context) { // this should still not be considered late testHarness.processElement( - new StreamRecord<>(WindowedValues.of(17, new Instant(0), window1, PaneInfo.NO_FIRING))); + new StreamRecord<>( + WindowedValues.of(17, Instant.ofEpochMilli(0), window1, PaneInfo.NO_FIRING))); assertThat( stripStreamRecordFromWindowedValue(testHarness.getOutput()), - contains(WindowedValues.of("17", new Instant(0), window1, PaneInfo.NO_FIRING))); + contains(WindowedValues.of("17", Instant.ofEpochMilli(0), window1, PaneInfo.NO_FIRING))); testHarness.getOutput().clear(); @@ -672,7 +675,8 @@ public void processElement(ProcessContext context) { // this should now be considered late testHarness.processElement( - new StreamRecord<>(WindowedValues.of(17, new Instant(0), window1, PaneInfo.NO_FIRING))); + new StreamRecord<>( + WindowedValues.of(17, Instant.ofEpochMilli(0), window1, PaneInfo.NO_FIRING))); assertThat(stripStreamRecordFromWindowedValue(testHarness.getOutput()), emptyIterable()); @@ -703,23 +707,25 @@ public void testStateGCForStatefulFn() throws Exception { assertEquals(0, testHarness.numKeyedStateEntries()); - IntervalWindow window1 = new IntervalWindow(new Instant(0), Duration.millis(10)); + IntervalWindow window1 = new IntervalWindow(Instant.ofEpochMilli(0), Duration.millis(10)); testHarness.processElement( new StreamRecord<>( - WindowedValues.of(KV.of("key1", 5), new Instant(1), window1, PaneInfo.NO_FIRING))); + WindowedValues.of( + KV.of("key1", 5), Instant.ofEpochMilli(1), window1, PaneInfo.NO_FIRING))); testHarness.processElement( new StreamRecord<>( - WindowedValues.of(KV.of("key2", 7), new Instant(3), window1, PaneInfo.NO_FIRING))); + WindowedValues.of( + KV.of("key2", 7), Instant.ofEpochMilli(3), window1, PaneInfo.NO_FIRING))); assertThat( stripStreamRecordFromWindowedValue(testHarness.getOutput()), contains( WindowedValues.of( - KV.of("key1", 5 + offset), new Instant(1), window1, PaneInfo.NO_FIRING), + KV.of("key1", 5 + offset), Instant.ofEpochMilli(1), window1, PaneInfo.NO_FIRING), WindowedValues.of( - KV.of("key2", 7 + offset), new Instant(3), window1, PaneInfo.NO_FIRING))); + KV.of("key2", 7 + offset), Instant.ofEpochMilli(3), window1, PaneInfo.NO_FIRING))); // 2 entries for the elements and 2 for the pending timers assertEquals(4, testHarness.numKeyedStateEntries()); @@ -742,9 +748,9 @@ public void testStateGCForStatefulFn() throws Exception { stripStreamRecordFromWindowedValue(testHarness.getOutput()), contains( WindowedValues.of( - KV.of("key1", timerOutput), new Instant(9), window1, PaneInfo.NO_FIRING), + KV.of("key1", timerOutput), Instant.ofEpochMilli(9), window1, PaneInfo.NO_FIRING), WindowedValues.of( - KV.of("key2", timerOutput), new Instant(9), window1, PaneInfo.NO_FIRING))); + KV.of("key2", timerOutput), Instant.ofEpochMilli(9), window1, PaneInfo.NO_FIRING))); testHarness.close(); } @@ -755,7 +761,8 @@ public void testGCForGlobalWindow() throws Exception { KeyedOneInputStreamOperatorTestHarness< FlinkKey, WindowedValue>, WindowedValue>> - testHarness = getHarness(windowingStrategy, 5000, (window) -> new Instant(50), 4092); + testHarness = + getHarness(windowingStrategy, 5000, (window) -> Instant.ofEpochMilli(50), 4092); testHarness.open(); @@ -768,11 +775,17 @@ public void testGCForGlobalWindow() throws Exception { testHarness.processElement( new StreamRecord<>( WindowedValues.of( - KV.of("key1", 5), new Instant(23), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING))); + KV.of("key1", 5), + Instant.ofEpochMilli(23), + GlobalWindow.INSTANCE, + PaneInfo.NO_FIRING))); testHarness.processElement( new StreamRecord<>( WindowedValues.of( - KV.of("key2", 6), new Instant(42), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING))); + KV.of("key2", 6), + Instant.ofEpochMilli(42), + GlobalWindow.INSTANCE, + PaneInfo.NO_FIRING))); // timers set by the transform assertThat(testHarness.numEventTimeTimers(), is(2)); @@ -805,7 +818,10 @@ public void testGCForGlobalWindow() throws Exception { testHarness.processElement( new StreamRecord<>( WindowedValues.of( - KV.of("key2", 6), new Instant(42), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING))); + KV.of("key2", 6), + Instant.ofEpochMilli(42), + GlobalWindow.INSTANCE, + PaneInfo.NO_FIRING))); // Close sends Flink's max watermark and will cleanup again testHarness.close(); @@ -944,8 +960,10 @@ outputTag, coder, new SerializablePipelineOptions(FlinkPipelineOptions.defaults( testHarness.open(); - IntervalWindow firstWindow = new IntervalWindow(new Instant(0), new Instant(100)); - IntervalWindow secondWindow = new IntervalWindow(new Instant(0), new Instant(500)); + IntervalWindow firstWindow = + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(100)); + IntervalWindow secondWindow = + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(500)); // test the keep of sideInputs events testHarness.processElement2( @@ -955,7 +973,7 @@ outputTag, coder, new SerializablePipelineOptions(FlinkPipelineOptions.defaults( valuesInWindow( PCollectionViewTesting.materializeValuesFor( view1.getPipeline().getOptions(), View.asIterable(), "hello", "ciao"), - new Instant(0), + Instant.ofEpochMilli(0), firstWindow)))); testHarness.processElement2( new StreamRecord<>( @@ -964,12 +982,14 @@ outputTag, coder, new SerializablePipelineOptions(FlinkPipelineOptions.defaults( valuesInWindow( PCollectionViewTesting.materializeValuesFor( view2.getPipeline().getOptions(), View.asIterable(), "foo", "bar"), - new Instant(0), + Instant.ofEpochMilli(0), secondWindow)))); // push in a regular elements - WindowedValue helloElement = valueInWindow("Hello", new Instant(0), firstWindow); - WindowedValue worldElement = valueInWindow("World", new Instant(1000), firstWindow); + WindowedValue helloElement = + valueInWindow("Hello", Instant.ofEpochMilli(0), firstWindow); + WindowedValue worldElement = + valueInWindow("World", Instant.ofEpochMilli(1000), firstWindow); testHarness.processElement1(new StreamRecord<>(helloElement)); testHarness.processElement1(new StreamRecord<>(worldElement)); @@ -981,7 +1001,7 @@ outputTag, coder, new SerializablePipelineOptions(FlinkPipelineOptions.defaults( valuesInWindow( PCollectionViewTesting.materializeValuesFor( view1.getPipeline().getOptions(), View.asIterable(), "hello", "ciao"), - new Instant(1000), + Instant.ofEpochMilli(1000), firstWindow)))); testHarness.processElement2( new StreamRecord<>( @@ -990,7 +1010,7 @@ outputTag, coder, new SerializablePipelineOptions(FlinkPipelineOptions.defaults( valuesInWindow( PCollectionViewTesting.materializeValuesFor( view2.getPipeline().getOptions(), View.asIterable(), "foo", "bar"), - new Instant(1000), + Instant.ofEpochMilli(1000), secondWindow)))); assertThat( @@ -1190,8 +1210,10 @@ void sideInputCheckpointing( testHarness.open(); - IntervalWindow firstWindow = new IntervalWindow(new Instant(0), new Instant(100)); - IntervalWindow secondWindow = new IntervalWindow(new Instant(0), new Instant(500)); + IntervalWindow firstWindow = + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(100)); + IntervalWindow secondWindow = + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(500)); // push in some side inputs for both windows testHarness.processElement2( @@ -1201,7 +1223,7 @@ void sideInputCheckpointing( valuesInWindow( PCollectionViewTesting.materializeValuesFor( view1.getPipeline().getOptions(), View.asIterable(), "hello", "ciao"), - new Instant(0), + Instant.ofEpochMilli(0), firstWindow)))); testHarness.processElement2( new StreamRecord<>( @@ -1210,7 +1232,7 @@ void sideInputCheckpointing( valuesInWindow( PCollectionViewTesting.materializeValuesFor( view2.getPipeline().getOptions(), View.asIterable(), "foo", "bar"), - new Instant(0), + Instant.ofEpochMilli(0), secondWindow)))); // snapshot state, throw away the operator, then restore and verify that we still match @@ -1223,8 +1245,10 @@ void sideInputCheckpointing( testHarness.open(); // push in main-input elements - WindowedValue helloElement = valueInWindow("Hello", new Instant(0), firstWindow); - WindowedValue worldElement = valueInWindow("World", new Instant(1000), firstWindow); + WindowedValue helloElement = + valueInWindow("Hello", Instant.ofEpochMilli(0), firstWindow); + WindowedValue worldElement = + valueInWindow("World", Instant.ofEpochMilli(1000), firstWindow); testHarness.processElement1(new StreamRecord<>(helloElement)); testHarness.processElement1(new StreamRecord<>(worldElement)); @@ -1336,12 +1360,16 @@ void pushbackDataCheckpointing( testHarness.open(); - IntervalWindow firstWindow = new IntervalWindow(new Instant(0), new Instant(100)); - IntervalWindow secondWindow = new IntervalWindow(new Instant(0), new Instant(500)); + IntervalWindow firstWindow = + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(100)); + IntervalWindow secondWindow = + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(500)); // push in main-input elements - WindowedValue helloElement = valueInWindow("Hello", new Instant(0), firstWindow); - WindowedValue worldElement = valueInWindow("World", new Instant(1000), firstWindow); + WindowedValue helloElement = + valueInWindow("Hello", Instant.ofEpochMilli(0), firstWindow); + WindowedValue worldElement = + valueInWindow("World", Instant.ofEpochMilli(1000), firstWindow); testHarness.processElement1(new StreamRecord<>(helloElement)); testHarness.processElement1(new StreamRecord<>(worldElement)); @@ -1362,7 +1390,7 @@ void pushbackDataCheckpointing( valuesInWindow( PCollectionViewTesting.materializeValuesFor( view1.getPipeline().getOptions(), View.asIterable(), "hello", "ciao"), - new Instant(0), + Instant.ofEpochMilli(0), firstWindow)))); testHarness.processElement2( new StreamRecord<>( @@ -1371,7 +1399,7 @@ void pushbackDataCheckpointing( valuesInWindow( PCollectionViewTesting.materializeValuesFor( view2.getPipeline().getOptions(), View.asIterable(), "foo", "bar"), - new Instant(0), + Instant.ofEpochMilli(0), secondWindow)))); assertThat( @@ -1383,7 +1411,7 @@ void pushbackDataCheckpointing( @Test public void testTimersRestore() throws Exception { - final Instant timerTimestamp = new Instant(1000); + final Instant timerTimestamp = Instant.ofEpochMilli(1000); final String outputMessage = "Timer fired"; WindowingStrategy windowingStrategy = @@ -1444,11 +1472,12 @@ public void onEventTime(OnTimerContext context) { testHarness.processWatermark(0); - IntervalWindow window1 = new IntervalWindow(new Instant(0), Duration.millis(10_000)); + IntervalWindow window1 = new IntervalWindow(Instant.ofEpochMilli(0), Duration.millis(10_000)); // this should register a timer testHarness.processElement( - new StreamRecord<>(WindowedValues.of(13, new Instant(0), window1, PaneInfo.NO_FIRING))); + new StreamRecord<>( + WindowedValues.of(13, Instant.ofEpochMilli(0), window1, PaneInfo.NO_FIRING))); assertThat(stripStreamRecordFromWindowedValue(testHarness.getOutput()), emptyIterable()); diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperatorTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperatorTest.java index 01c9d25f1bf1..6350bd1ee093 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperatorTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperatorTest.java @@ -498,7 +498,8 @@ public void testWatermarkHandling() throws Exception { assertThat(operator.getCurrentOutputWatermark(), is(0L)); // Trigger a new bundle - IntervalWindow intervalWindow = new IntervalWindow(new Instant(0), new Instant(9)); + IntervalWindow intervalWindow = + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(9)); WindowedValue> windowedValue = WindowedValues.of(KV.of("one", 1), Instant.now(), intervalWindow, PaneInfo.NO_FIRING); testHarness.processElement(new StreamRecord<>(windowedValue)); @@ -522,8 +523,8 @@ public void testWatermarkHandling() throws Exception { assertThat(testHarness.numEventTimeTimers(), is(1)); // cleanup timer // Set at timer - Instant timerTarget = new Instant(5); - Instant timerTarget2 = new Instant(6); + Instant timerTarget = Instant.ofEpochMilli(5); + Instant timerTarget2 = Instant.ofEpochMilli(6); operator.getLockToAcquireForStateAccessDuringBundles().lock(); BiConsumer timerConsumer = @@ -578,7 +579,7 @@ public void testWatermarkHandling() throws Exception { // Watermark is advanced in a blocking fashion on close, not via a timers // Create a bundle with a pending timer to simulate that testHarness.processElement(new StreamRecord<>(windowedValue)); - timerConsumer.accept("timer3", new Instant(targetWatermark)); + timerConsumer.accept("timer3", Instant.ofEpochMilli(targetWatermark)); assertThat(testHarness.numEventTimeTimers(), is(1)); // This should be blocking until the watermark reaches Long.MAX_VALUE. @@ -674,7 +675,7 @@ public void testEnsureStateCleanupWithKeyedInputCleanupTimer() { KeyedStateBackend keyedStateBackend = Mockito.mock(KeyedStateBackend.class); Lock stateBackendLock = Mockito.mock(Lock.class); StringUtf8Coder keyCoder = StringUtf8Coder.of(); - IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(10)); + IntervalWindow window = new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10)); Coder windowCoder = IntervalWindow.getCoder(); // Test that cleanup timer is set correctly @@ -772,7 +773,7 @@ private void testEnsureDeferredStateCleanupTimerFiring(boolean withCheckpointing KV timerInputKey = KV.of("transformId", "timerId"); AtomicBoolean timerInputReceived = new AtomicBoolean(); - IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(1000)); + IntervalWindow window = new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(1000)); IntervalWindow.IntervalWindowCoder windowCoder = IntervalWindow.IntervalWindowCoder.of(); WindowedValue> windowedValue = WindowedValues.of( diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperatorTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperatorTest.java index 3bee828f23dd..c935e082a209 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperatorTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperatorTest.java @@ -80,7 +80,7 @@ public void testRestore() throws Exception { testHarness.open(); // process elements - IntervalWindow window = new IntervalWindow(new Instant(0), Duration.millis(10_000)); + IntervalWindow window = new IntervalWindow(Instant.ofEpochMilli(0), Duration.millis(10_000)); testHarness.processWatermark(0L); testHarness.processElement( Item.builder().key(1L).timestamp(1L).value(100L).window(window).build().toStreamRecord()); @@ -110,12 +110,12 @@ public void testRestore() throws Exception { containsInAnyOrder( WindowedValues.of( KV.of(1L, 120L), - new Instant(9_999), + Instant.ofEpochMilli(9_999), window, PaneInfo.createPane(true, true, ON_TIME)), WindowedValues.of( KV.of(2L, 77L), - new Instant(9_999), + Instant.ofEpochMilli(9_999), window, PaneInfo.createPane(true, true, ON_TIME)))); // cleanup @@ -135,8 +135,8 @@ public void testTimerCleanupOfPendingTimerList() throws Exception { timerInternals = windowDoFnOperator.timerInternals; // process elements - IntervalWindow window = new IntervalWindow(new Instant(0), Duration.millis(100)); - IntervalWindow window2 = new IntervalWindow(new Instant(100), Duration.millis(100)); + IntervalWindow window = new IntervalWindow(Instant.ofEpochMilli(0), Duration.millis(100)); + IntervalWindow window2 = new IntervalWindow(Instant.ofEpochMilli(100), Duration.millis(100)); testHarness.processWatermark(0L); // Use two different keys to check for correct watermark hold calculation @@ -186,10 +186,13 @@ public void testTimerCleanupOfPendingTimerList() throws Exception { stripStreamRecordFromWindowedValue(testHarness.getOutput()), containsInAnyOrder( WindowedValues.of( - KV.of(1L, 100L), new Instant(99), window, PaneInfo.createPane(true, true, ON_TIME)), + KV.of(1L, 100L), + Instant.ofEpochMilli(99), + window, + PaneInfo.createPane(true, true, ON_TIME)), WindowedValues.of( KV.of(2L, 150L), - new Instant(199), + Instant.ofEpochMilli(199), window2, PaneInfo.createPane(true, true, ON_TIME)))); @@ -266,7 +269,7 @@ static ItemBuilder builder() { StreamRecord>> toStreamRecord() { WindowedValue> keyedItem = - WindowedValues.of(KV.of(key, value), new Instant(timestamp), window, NO_FIRING); + WindowedValues.of(KV.of(key, value), Instant.ofEpochMilli(timestamp), window, NO_FIRING); return new StreamRecord<>(keyedItem); } diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/TestCountingSource.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/TestCountingSource.java index 3af9062ba9b4..ff0e130d7b87 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/TestCountingSource.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/TestCountingSource.java @@ -238,7 +238,7 @@ public KV getCurrent() { @Override public Instant getCurrentTimestamp() { - return new Instant(current); + return Instant.ofEpochMilli(current); } @Override @@ -269,7 +269,7 @@ public Instant getWatermark() { // The watermark is a promise about future elements, and the timestamps of elements are // strictly increasing for this source. - return new Instant(current + 1); + return Instant.ofEpochMilli(current + 1); } @Override diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapperTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapperTest.java index f57198e08e3e..89b0de1852dc 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapperTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapperTest.java @@ -952,7 +952,7 @@ public Instant getWatermark() { if (currentIdx >= data.size() - 1) { return BoundedWindow.TIMESTAMP_MAX_VALUE; } - return new Instant(currentIdx); + return Instant.ofEpochMilli(currentIdx); } @Override @@ -976,7 +976,7 @@ public T getCurrent() throws NoSuchElementException { @Override public Instant getCurrentTimestamp() throws NoSuchElementException { if (currentIdx >= 0 && currentIdx < data.size()) { - return new Instant(currentIdx); + return Instant.ofEpochMilli(currentIdx); } throw new NoSuchElementException(); } diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferedElementsTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferedElementsTest.java index 8e8db7496b80..9ddfcaea1669 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferedElementsTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferedElementsTest.java @@ -53,15 +53,16 @@ public void testCoder() throws IOException { BufferedElement element = new BufferedElements.Element( - WindowedValues.of("test", new Instant(2), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); + WindowedValues.of( + "test", Instant.ofEpochMilli(2), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); BufferedElement timerElement = new BufferedElements.Timer( "timerId", "timerId", key, GlobalWindow.INSTANCE, - new Instant(1), - new Instant(1), + Instant.ofEpochMilli(1), + Instant.ofEpochMilli(1), TimeDomain.EVENT_TIME); testRoundTrip(ImmutableList.of(element), coder); diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/MonitoringUtil.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/MonitoringUtil.java index 0237923e4e47..c84c3c9784dc 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/MonitoringUtil.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/MonitoringUtil.java @@ -130,7 +130,7 @@ public int compare(JobMessage o1, JobMessage o2) { */ public List getJobMessages(String jobId, long startTimestampMs) throws IOException { // TODO: Allow filtering messages by importance - Instant startTimestamp = new Instant(startTimestampMs); + Instant startTimestamp = Instant.ofEpochMilli(startTimestampMs); ArrayList allMessages = new ArrayList<>(); String pageToken = null; while (true) { diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BatchViewOverridesTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BatchViewOverridesTest.java index 1f160ffdf3af..8db9a819bd60 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BatchViewOverridesTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BatchViewOverridesTest.java @@ -122,9 +122,9 @@ public void testBatchViewAsListToIsmRecordForNonGlobalWindow() throws Exception new BatchViewOverrides.BatchViewAsList.ToIsmRecordForNonGlobalWindowDoFn< Long, IntervalWindow>(IntervalWindow.getCoder())); - IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10)); - IntervalWindow windowB = new IntervalWindow(new Instant(10), new Instant(20)); - IntervalWindow windowC = new IntervalWindow(new Instant(20), new Instant(30)); + IntervalWindow windowA = new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10)); + IntervalWindow windowB = new IntervalWindow(Instant.ofEpochMilli(10), Instant.ofEpochMilli(20)); + IntervalWindow windowC = new IntervalWindow(Instant.ofEpochMilli(20), Instant.ofEpochMilli(30)); Iterable>>>> inputElements = ImmutableList.of( @@ -134,20 +134,24 @@ public void testBatchViewAsListToIsmRecordForNonGlobalWindow() throws Exception ImmutableList.of( KV.of( windowA, - WindowedValues.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)), + WindowedValues.of( + 110L, Instant.ofEpochMilli(1), windowA, PaneInfo.NO_FIRING)), KV.of( windowA, - WindowedValues.of(111L, new Instant(3), windowA, PaneInfo.NO_FIRING)), + WindowedValues.of( + 111L, Instant.ofEpochMilli(3), windowA, PaneInfo.NO_FIRING)), KV.of( windowA, - WindowedValues.of(112L, new Instant(4), windowA, PaneInfo.NO_FIRING)), + WindowedValues.of( + 112L, Instant.ofEpochMilli(4), windowA, PaneInfo.NO_FIRING)), KV.of( windowB, - WindowedValues.of(120L, new Instant(12), windowB, PaneInfo.NO_FIRING)), + WindowedValues.of( + 120L, Instant.ofEpochMilli(12), windowB, PaneInfo.NO_FIRING)), KV.of( windowB, WindowedValues.of( - 121L, new Instant(14), windowB, PaneInfo.NO_FIRING)))), + 121L, Instant.ofEpochMilli(14), windowB, PaneInfo.NO_FIRING)))), KV.of( 2, (Iterable>>) @@ -155,7 +159,7 @@ public void testBatchViewAsListToIsmRecordForNonGlobalWindow() throws Exception KV.of( windowC, WindowedValues.of( - 210L, new Instant(25), windowC, PaneInfo.NO_FIRING))))); + 210L, Instant.ofEpochMilli(25), windowC, PaneInfo.NO_FIRING))))); // The order of the output elements is important relative to processing order assertThat( @@ -163,22 +167,22 @@ public void testBatchViewAsListToIsmRecordForNonGlobalWindow() throws Exception contains( IsmRecord.of( ImmutableList.of(windowA, 0L), - WindowedValues.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)), + WindowedValues.of(110L, Instant.ofEpochMilli(1), windowA, PaneInfo.NO_FIRING)), IsmRecord.of( ImmutableList.of(windowA, 1L), - WindowedValues.of(111L, new Instant(3), windowA, PaneInfo.NO_FIRING)), + WindowedValues.of(111L, Instant.ofEpochMilli(3), windowA, PaneInfo.NO_FIRING)), IsmRecord.of( ImmutableList.of(windowA, 2L), - WindowedValues.of(112L, new Instant(4), windowA, PaneInfo.NO_FIRING)), + WindowedValues.of(112L, Instant.ofEpochMilli(4), windowA, PaneInfo.NO_FIRING)), IsmRecord.of( ImmutableList.of(windowB, 0L), - WindowedValues.of(120L, new Instant(12), windowB, PaneInfo.NO_FIRING)), + WindowedValues.of(120L, Instant.ofEpochMilli(12), windowB, PaneInfo.NO_FIRING)), IsmRecord.of( ImmutableList.of(windowB, 1L), - WindowedValues.of(121L, new Instant(14), windowB, PaneInfo.NO_FIRING)), + WindowedValues.of(121L, Instant.ofEpochMilli(14), windowB, PaneInfo.NO_FIRING)), IsmRecord.of( ImmutableList.of(windowC, 0L), - WindowedValues.of(210L, new Instant(25), windowC, PaneInfo.NO_FIRING)))); + WindowedValues.of(210L, Instant.ofEpochMilli(25), windowC, PaneInfo.NO_FIRING)))); } @Test @@ -210,9 +214,9 @@ public void testToIsmRecordForMapLikeDoFn() throws Exception { ismCoder, false /* unique keys */)); - IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10)); - IntervalWindow windowB = new IntervalWindow(new Instant(10), new Instant(20)); - IntervalWindow windowC = new IntervalWindow(new Instant(20), new Instant(30)); + IntervalWindow windowA = new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10)); + IntervalWindow windowB = new IntervalWindow(Instant.ofEpochMilli(10), Instant.ofEpochMilli(20)); + IntervalWindow windowC = new IntervalWindow(Instant.ofEpochMilli(20), Instant.ofEpochMilli(30)); Iterable, WindowedValue>>>> inputElements = @@ -224,27 +228,27 @@ public void testToIsmRecordForMapLikeDoFn() throws Exception { KV.of( KV.of(1L, windowA), WindowedValues.of( - 110L, new Instant(1), windowA, PaneInfo.NO_FIRING)), + 110L, Instant.ofEpochMilli(1), windowA, PaneInfo.NO_FIRING)), // same window same key as to previous KV.of( KV.of(1L, windowA), WindowedValues.of( - 111L, new Instant(2), windowA, PaneInfo.NO_FIRING)), + 111L, Instant.ofEpochMilli(2), windowA, PaneInfo.NO_FIRING)), // same window different key as to previous KV.of( KV.of(2L, windowA), WindowedValues.of( - 120L, new Instant(3), windowA, PaneInfo.NO_FIRING)), + 120L, Instant.ofEpochMilli(3), windowA, PaneInfo.NO_FIRING)), // different window same key as to previous KV.of( KV.of(2L, windowB), WindowedValues.of( - 210L, new Instant(11), windowB, PaneInfo.NO_FIRING)), + 210L, Instant.ofEpochMilli(11), windowB, PaneInfo.NO_FIRING)), // different window and different key as to previous KV.of( KV.of(3L, windowB), WindowedValues.of( - 220L, new Instant(12), windowB, PaneInfo.NO_FIRING)))), + 220L, Instant.ofEpochMilli(12), windowB, PaneInfo.NO_FIRING)))), KV.of( 2, (Iterable, WindowedValue>>) @@ -253,7 +257,10 @@ public void testToIsmRecordForMapLikeDoFn() throws Exception { KV.of( KV.of(4L, windowC), WindowedValues.of( - 330L, new Instant(21), windowC, PaneInfo.NO_FIRING))))); + 330L, + Instant.ofEpochMilli(21), + windowC, + PaneInfo.NO_FIRING))))); // The order of the output elements is important relative to processing order assertThat( @@ -261,22 +268,22 @@ public void testToIsmRecordForMapLikeDoFn() throws Exception { contains( IsmRecord.of( ImmutableList.of(1L, windowA, 0L), - WindowedValues.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)), + WindowedValues.of(110L, Instant.ofEpochMilli(1), windowA, PaneInfo.NO_FIRING)), IsmRecord.of( ImmutableList.of(1L, windowA, 1L), - WindowedValues.of(111L, new Instant(2), windowA, PaneInfo.NO_FIRING)), + WindowedValues.of(111L, Instant.ofEpochMilli(2), windowA, PaneInfo.NO_FIRING)), IsmRecord.of( ImmutableList.of(2L, windowA, 0L), - WindowedValues.of(120L, new Instant(3), windowA, PaneInfo.NO_FIRING)), + WindowedValues.of(120L, Instant.ofEpochMilli(3), windowA, PaneInfo.NO_FIRING)), IsmRecord.of( ImmutableList.of(2L, windowB, 0L), - WindowedValues.of(210L, new Instant(11), windowB, PaneInfo.NO_FIRING)), + WindowedValues.of(210L, Instant.ofEpochMilli(11), windowB, PaneInfo.NO_FIRING)), IsmRecord.of( ImmutableList.of(3L, windowB, 0L), - WindowedValues.of(220L, new Instant(12), windowB, PaneInfo.NO_FIRING)), + WindowedValues.of(220L, Instant.ofEpochMilli(12), windowB, PaneInfo.NO_FIRING)), IsmRecord.of( ImmutableList.of(4L, windowC, 0L), - WindowedValues.of(330L, new Instant(21), windowC, PaneInfo.NO_FIRING)))); + WindowedValues.of(330L, Instant.ofEpochMilli(21), windowC, PaneInfo.NO_FIRING)))); // Verify the number of unique keys per window. assertThat( @@ -342,7 +349,7 @@ public void testToIsmRecordForMapLikeDoFnWithoutUniqueKeysThrowsException() thro ismCoder, true /* unique keys */)); - IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10)); + IntervalWindow windowA = new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10)); Iterable, WindowedValue>>>> inputElements = @@ -354,12 +361,12 @@ public void testToIsmRecordForMapLikeDoFnWithoutUniqueKeysThrowsException() thro KV.of( KV.of(1L, windowA), WindowedValues.of( - 110L, new Instant(1), windowA, PaneInfo.NO_FIRING)), + 110L, Instant.ofEpochMilli(1), windowA, PaneInfo.NO_FIRING)), // same window same key as to previous KV.of( KV.of(1L, windowA), WindowedValues.of( - 111L, new Instant(2), windowA, PaneInfo.NO_FIRING))))); + 111L, Instant.ofEpochMilli(2), windowA, PaneInfo.NO_FIRING))))); thrown.expect(IllegalStateException.class); thrown.expectMessage("Unique keys are expected but found key"); @@ -386,9 +393,9 @@ public void testToIsmMetadataRecordForSizeDoFn() throws Exception { new BatchViewOverrides.BatchViewAsMultimap.ToIsmMetadataRecordForSizeDoFn< Long, Long, IntervalWindow>(windowCoder)); - IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10)); - IntervalWindow windowB = new IntervalWindow(new Instant(10), new Instant(20)); - IntervalWindow windowC = new IntervalWindow(new Instant(20), new Instant(30)); + IntervalWindow windowA = new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10)); + IntervalWindow windowB = new IntervalWindow(Instant.ofEpochMilli(10), Instant.ofEpochMilli(20)); + IntervalWindow windowC = new IntervalWindow(Instant.ofEpochMilli(20), Instant.ofEpochMilli(30)); Iterable>>> inputElements = ImmutableList.of( @@ -435,9 +442,9 @@ public void testToIsmMetadataRecordForKeyDoFn() throws Exception { new BatchViewOverrides.BatchViewAsMultimap.ToIsmMetadataRecordForKeyDoFn< Long, Long, IntervalWindow>(keyCoder, windowCoder)); - IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10)); - IntervalWindow windowB = new IntervalWindow(new Instant(10), new Instant(20)); - IntervalWindow windowC = new IntervalWindow(new Instant(20), new Instant(30)); + IntervalWindow windowA = new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10)); + IntervalWindow windowB = new IntervalWindow(Instant.ofEpochMilli(10), Instant.ofEpochMilli(20)); + IntervalWindow windowC = new IntervalWindow(Instant.ofEpochMilli(20), Instant.ofEpochMilli(30)); Iterable>>> inputElements = ImmutableList.of( @@ -484,9 +491,9 @@ public void testToMapDoFn() throws Exception { new BatchViewOverrides.BatchViewAsMap.ToMapDoFn( windowCoder)); - IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10)); - IntervalWindow windowB = new IntervalWindow(new Instant(10), new Instant(20)); - IntervalWindow windowC = new IntervalWindow(new Instant(20), new Instant(30)); + IntervalWindow windowA = new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10)); + IntervalWindow windowB = new IntervalWindow(Instant.ofEpochMilli(10), Instant.ofEpochMilli(20)); + IntervalWindow windowC = new IntervalWindow(Instant.ofEpochMilli(20), Instant.ofEpochMilli(30)); Iterable>>>>> inputElements = @@ -498,20 +505,29 @@ public void testToMapDoFn() throws Exception { KV.of( windowA, WindowedValues.of( - KV.of(1L, 11L), new Instant(3), windowA, PaneInfo.NO_FIRING)), + KV.of(1L, 11L), + Instant.ofEpochMilli(3), + windowA, + PaneInfo.NO_FIRING)), KV.of( windowA, WindowedValues.of( - KV.of(2L, 21L), new Instant(7), windowA, PaneInfo.NO_FIRING)), + KV.of(2L, 21L), + Instant.ofEpochMilli(7), + windowA, + PaneInfo.NO_FIRING)), KV.of( windowB, WindowedValues.of( - KV.of(2L, 21L), new Instant(13), windowB, PaneInfo.NO_FIRING)), + KV.of(2L, 21L), + Instant.ofEpochMilli(13), + windowB, + PaneInfo.NO_FIRING)), KV.of( windowB, WindowedValues.of( KV.of(3L, 31L), - new Instant(15), + Instant.ofEpochMilli(15), windowB, PaneInfo.NO_FIRING)))), KV.of( @@ -522,7 +538,7 @@ public void testToMapDoFn() throws Exception { windowC, WindowedValues.of( KV.of(4L, 41L), - new Instant(25), + Instant.ofEpochMilli(25), windowC, PaneInfo.NO_FIRING))))); @@ -558,9 +574,9 @@ public void testToMultimapDoFn() throws Exception { new BatchViewOverrides.BatchViewAsMultimap.ToMultimapDoFn< Long, Long, IntervalWindow>(windowCoder)); - IntervalWindow windowA = new IntervalWindow(new Instant(0), new Instant(10)); - IntervalWindow windowB = new IntervalWindow(new Instant(10), new Instant(20)); - IntervalWindow windowC = new IntervalWindow(new Instant(20), new Instant(30)); + IntervalWindow windowA = new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10)); + IntervalWindow windowB = new IntervalWindow(Instant.ofEpochMilli(10), Instant.ofEpochMilli(20)); + IntervalWindow windowC = new IntervalWindow(Instant.ofEpochMilli(20), Instant.ofEpochMilli(30)); Iterable>>>>> inputElements = @@ -572,30 +588,45 @@ public void testToMultimapDoFn() throws Exception { KV.of( windowA, WindowedValues.of( - KV.of(1L, 11L), new Instant(3), windowA, PaneInfo.NO_FIRING)), + KV.of(1L, 11L), + Instant.ofEpochMilli(3), + windowA, + PaneInfo.NO_FIRING)), // [BEAM-5184] Specifically test with a duplicate value to ensure that // duplicate key/values are not lost. KV.of( windowA, WindowedValues.of( - KV.of(1L, 11L), new Instant(3), windowA, PaneInfo.NO_FIRING)), + KV.of(1L, 11L), + Instant.ofEpochMilli(3), + windowA, + PaneInfo.NO_FIRING)), KV.of( windowA, WindowedValues.of( - KV.of(1L, 12L), new Instant(5), windowA, PaneInfo.NO_FIRING)), + KV.of(1L, 12L), + Instant.ofEpochMilli(5), + windowA, + PaneInfo.NO_FIRING)), KV.of( windowA, WindowedValues.of( - KV.of(2L, 21L), new Instant(7), windowA, PaneInfo.NO_FIRING)), + KV.of(2L, 21L), + Instant.ofEpochMilli(7), + windowA, + PaneInfo.NO_FIRING)), KV.of( windowB, WindowedValues.of( - KV.of(2L, 21L), new Instant(13), windowB, PaneInfo.NO_FIRING)), + KV.of(2L, 21L), + Instant.ofEpochMilli(13), + windowB, + PaneInfo.NO_FIRING)), KV.of( windowB, WindowedValues.of( KV.of(3L, 31L), - new Instant(15), + Instant.ofEpochMilli(15), windowB, PaneInfo.NO_FIRING)))), KV.of( @@ -606,7 +637,7 @@ public void testToMultimapDoFn() throws Exception { windowC, WindowedValues.of( KV.of(4L, 41L), - new Instant(25), + Instant.ofEpochMilli(25), windowC, PaneInfo.NO_FIRING))))); diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java index 54ba10df9d1c..a22cdd506afe 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java @@ -497,7 +497,7 @@ public void testWaitUntilFinishNoRepeatedLogs() throws Exception { Sleeper sleeper = new ZeroSleeper(); NanoClock nanoClock = mock(NanoClock.class); - Instant separatingTimestamp = new Instant(42L); + Instant separatingTimestamp = Instant.ofEpochMilli(42L); JobMessage theMessage = infoMessage(separatingTimestamp, "nothing"); MonitoringUtil mockMonitor = mock(MonitoringUtil.class); diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/MonitoringUtilTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/MonitoringUtilTest.java index 5f76b6750ffa..30d0cc6c544f 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/MonitoringUtilTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/MonitoringUtilTest.java @@ -62,7 +62,7 @@ public void testGetJobMessages() throws IOException { for (long i = 0; i < 100; ++i) { JobMessage message = new JobMessage(); message.setId("message_" + i); - message.setTime(TimeUtil.toCloudTime(new Instant(i))); + message.setTime(TimeUtil.toCloudTime(Instant.ofEpochMilli(i))); firstResponse.getJobMessages().add(message); } String pageToken = "page_token"; @@ -73,7 +73,7 @@ public void testGetJobMessages() throws IOException { for (long i = 100; i < 150; ++i) { JobMessage message = new JobMessage(); message.setId("message_" + i); - message.setTime(TimeUtil.toCloudTime(new Instant(i))); + message.setTime(TimeUtil.toCloudTime(Instant.ofEpochMilli(i))); secondResponse.getJobMessages().add(message); } diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/TimeUtilTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/TimeUtilTest.java index 1ac9fabf6a45..a9c0860fca3b 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/TimeUtilTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/TimeUtilTest.java @@ -35,23 +35,23 @@ public final class TimeUtilTest { @Test public void toCloudTimeShouldPrintTimeStrings() { - assertEquals("1970-01-01T00:00:00Z", toCloudTime(new Instant(0))); - assertEquals("1970-01-01T00:00:00.001Z", toCloudTime(new Instant(1))); + assertEquals("1970-01-01T00:00:00Z", toCloudTime(Instant.ofEpochMilli(0))); + assertEquals("1970-01-01T00:00:00.001Z", toCloudTime(Instant.ofEpochMilli(1))); } @Test public void fromCloudTimeShouldParseTimeStrings() { - assertEquals(new Instant(0), fromCloudTime("1970-01-01T00:00:00Z")); - assertEquals(new Instant(1), fromCloudTime("1970-01-01T00:00:00.001Z")); - assertEquals(new Instant(1), fromCloudTime("1970-01-01T00:00:00.001000Z")); - assertEquals(new Instant(1), fromCloudTime("1970-01-01T00:00:00.001001Z")); - assertEquals(new Instant(1), fromCloudTime("1970-01-01T00:00:00.001000000Z")); - assertEquals(new Instant(1), fromCloudTime("1970-01-01T00:00:00.001000001Z")); - assertEquals(new Instant(0), fromCloudTime("1970-01-01T00:00:00.0Z")); - assertEquals(new Instant(0), fromCloudTime("1970-01-01T00:00:00.00Z")); - assertEquals(new Instant(420), fromCloudTime("1970-01-01T00:00:00.42Z")); - assertEquals(new Instant(300), fromCloudTime("1970-01-01T00:00:00.3Z")); - assertEquals(new Instant(20), fromCloudTime("1970-01-01T00:00:00.02Z")); + assertEquals(Instant.ofEpochMilli(0), fromCloudTime("1970-01-01T00:00:00Z")); + assertEquals(Instant.ofEpochMilli(1), fromCloudTime("1970-01-01T00:00:00.001Z")); + assertEquals(Instant.ofEpochMilli(1), fromCloudTime("1970-01-01T00:00:00.001000Z")); + assertEquals(Instant.ofEpochMilli(1), fromCloudTime("1970-01-01T00:00:00.001001Z")); + assertEquals(Instant.ofEpochMilli(1), fromCloudTime("1970-01-01T00:00:00.001000000Z")); + assertEquals(Instant.ofEpochMilli(1), fromCloudTime("1970-01-01T00:00:00.001000001Z")); + assertEquals(Instant.ofEpochMilli(0), fromCloudTime("1970-01-01T00:00:00.0Z")); + assertEquals(Instant.ofEpochMilli(0), fromCloudTime("1970-01-01T00:00:00.00Z")); + assertEquals(Instant.ofEpochMilli(420), fromCloudTime("1970-01-01T00:00:00.42Z")); + assertEquals(Instant.ofEpochMilli(300), fromCloudTime("1970-01-01T00:00:00.3Z")); + assertEquals(Instant.ofEpochMilli(20), fromCloudTime("1970-01-01T00:00:00.02Z")); assertNull(fromCloudTime("")); assertNull(fromCloudTime("1970-01-01T00:00:00")); assertNull(fromCloudTime("1970-01-01T00:00:00.1e3Z")); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillTimeUtils.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillTimeUtils.java index e3ceddda6375..157553dc48c6 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillTimeUtils.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillTimeUtils.java @@ -44,7 +44,7 @@ public class WindmillTimeUtils { public static Instant windmillToHarnessTimestamp(long timestampUs) { // Windmill should never send us an unknown timestamp. Preconditions.checkArgument(timestampUs != Long.MIN_VALUE); - Instant result = new Instant(divideAndRoundDown(timestampUs, 1000)); + Instant result = Instant.ofEpochMilli(divideAndRoundDown(timestampUs, 1000)); if (result.isBefore(BoundedWindow.TIMESTAMP_MIN_VALUE)) { return BoundedWindow.TIMESTAMP_MIN_VALUE; } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/IdTracker.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/IdTracker.java index bbcf108b317e..5a9b4e64ef32 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/IdTracker.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/IdTracker.java @@ -117,7 +117,8 @@ Map, RangeSet> newSortedRangeMap() { private Range getTrackedRange(Instant ts) { Instant snapped = - new Instant(ts.getMillis() - ts.plus(RESOLUTION).getMillis() % RESOLUTION.getMillis()); + Instant.ofEpochMilli( + ts.getMillis() - ts.plus(RESOLUTION).getMillis() % RESOLUTION.getMillis()); return Range.closedOpen(snapped, snapped.plus(RESOLUTION)); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReader.java index c609bed4eae0..9f37361209bc 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReader.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReader.java @@ -832,7 +832,7 @@ private void consumeWatermark(Windmill.WatermarkHold watermarkHold, StateTag>>> NO_KVS = Collections.emptyList(); - private static final Instant timestamp = new Instant(123000); + private static final Instant timestamp = Instant.ofEpochMilli(123000); private static final IntervalWindow window = new IntervalWindow(timestamp, timestamp.plus(Duration.millis(1000))); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmSideInputReaderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmSideInputReaderTest.java index 06e089807299..892fee95e0c8 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmSideInputReaderTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmSideInputReaderTest.java @@ -317,23 +317,26 @@ public void testSingletonMap() throws Exception { @Test public void testSingletonMapInWindow() throws Exception { - IntervalWindow firstWindow = new IntervalWindow(new Instant(0L), new Instant(100L)); - IntervalWindow secondWindow = new IntervalWindow(new Instant(50L), new Instant(150L)); - IntervalWindow emptyWindow = new IntervalWindow(new Instant(75L), new Instant(175L)); + IntervalWindow firstWindow = + new IntervalWindow(Instant.ofEpochMilli(0L), Instant.ofEpochMilli(100L)); + IntervalWindow secondWindow = + new IntervalWindow(Instant.ofEpochMilli(50L), Instant.ofEpochMilli(150L)); + IntervalWindow emptyWindow = + new IntervalWindow(Instant.ofEpochMilli(75L), Instant.ofEpochMilli(175L)); final Map>> elements = ImmutableMap.>>builder() .put( firstWindow, WindowedValues.of( ImmutableMap.builder().put("foo", 0L).put("bar", -1L).build(), - new Instant(7), + Instant.ofEpochMilli(7), firstWindow, PaneInfo.NO_FIRING)) .put( secondWindow, WindowedValues.of( ImmutableMap.builder().put("bar", -1L).put("baz", 1L).build(), - new Instant(53L), + Instant.ofEpochMilli(53L), secondWindow, PaneInfo.NO_FIRING)) .build(); @@ -395,9 +398,12 @@ public void testSingletonMapInWindow() throws Exception { @Test public void testSingletonMultimapInWindow() throws Exception { - IntervalWindow firstWindow = new IntervalWindow(new Instant(0L), new Instant(100L)); - IntervalWindow secondWindow = new IntervalWindow(new Instant(50L), new Instant(150L)); - IntervalWindow emptyWindow = new IntervalWindow(new Instant(75L), new Instant(175L)); + IntervalWindow firstWindow = + new IntervalWindow(Instant.ofEpochMilli(0L), Instant.ofEpochMilli(100L)); + IntervalWindow secondWindow = + new IntervalWindow(Instant.ofEpochMilli(50L), Instant.ofEpochMilli(150L)); + IntervalWindow emptyWindow = + new IntervalWindow(Instant.ofEpochMilli(75L), Instant.ofEpochMilli(175L)); @SuppressWarnings({"unchecked", "rawtypes"}) // Collection is iterable, and this is immutable final Map>>> elements = ImmutableMap.>>>builder() @@ -411,7 +417,7 @@ public void testSingletonMultimapInWindow() throws Exception { .put("bar", -1L) .build() .asMap(), - new Instant(7), + Instant.ofEpochMilli(7), firstWindow, PaneInfo.NO_FIRING)) .put( @@ -424,7 +430,7 @@ public void testSingletonMultimapInWindow() throws Exception { .put("baz", 3L) .build() .asMap(), - new Instant(53L), + Instant.ofEpochMilli(53L), secondWindow, PaneInfo.NO_FIRING)) .build(); @@ -1628,11 +1634,14 @@ private static void verifyMap( WindowedValue valueInIntervalWindow(long value, long startOfWindow) { return WindowedValues.of( - value, new Instant(startOfWindow), intervalWindow(startOfWindow), PaneInfo.NO_FIRING); + value, + Instant.ofEpochMilli(startOfWindow), + intervalWindow(startOfWindow), + PaneInfo.NO_FIRING); } private static IntervalWindow intervalWindow(long start) { - return new IntervalWindow(new Instant(start), new Instant(start + 1)); + return new IntervalWindow(Instant.ofEpochMilli(start), Instant.ofEpochMilli(start + 1)); } private static BoundedWindow windowOf(WindowedValue windowedValue) { diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PartitioningShuffleReaderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PartitioningShuffleReaderTest.java index 9fe44e5789eb..a49b6864f83f 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PartitioningShuffleReaderTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PartitioningShuffleReaderTest.java @@ -50,7 +50,7 @@ public class PartitioningShuffleReaderTest { private static final List>> NO_KVS = Collections.emptyList(); - private static final Instant timestamp = new Instant(123000); + private static final Instant timestamp = Instant.ofEpochMilli(123000); private static final IntervalWindow window = new IntervalWindow(timestamp, timestamp.plus(Duration.millis(1000))); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubDynamicSinkTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubDynamicSinkTest.java index d8822ce4937b..33618dffc887 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubDynamicSinkTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubDynamicSinkTest.java @@ -121,15 +121,16 @@ public void testWriteDynamicDestinations() throws Exception { .build()); writer.add( WindowedValues.timestampedValueInGlobalWindow( - new PubsubMessage(payload1, null).withTopic("topic1"), new Instant(baseTimestamp))); + new PubsubMessage(payload1, null).withTopic("topic1"), + Instant.ofEpochMilli(baseTimestamp))); writer.add( WindowedValues.timestampedValueInGlobalWindow( new PubsubMessage(payload2, null).withTopic("topic2"), - new Instant(baseTimestamp + 1))); + Instant.ofEpochMilli(baseTimestamp + 1))); writer.add( WindowedValues.timestampedValueInGlobalWindow( new PubsubMessage(payload3, null).withTopic("topic3"), - new Instant(baseTimestamp + 2))); + Instant.ofEpochMilli(baseTimestamp + 2))); } writer.close(); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubReaderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubReaderTest.java index c28fdac04888..2fb0ede2b108 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubReaderTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubReaderTest.java @@ -93,13 +93,16 @@ private void testReadWith(String parseFn) throws Exception { NativeReader.NativeReaderIterator> iter = reader.iterator(); assertTrue(iter.start()); assertEquals( - iter.getCurrent(), WindowedValues.timestampedValueInGlobalWindow("e0", new Instant(0))); + iter.getCurrent(), + WindowedValues.timestampedValueInGlobalWindow("e0", Instant.ofEpochMilli(0))); assertTrue(iter.advance()); assertEquals( - iter.getCurrent(), WindowedValues.timestampedValueInGlobalWindow("e1", new Instant(1))); + iter.getCurrent(), + WindowedValues.timestampedValueInGlobalWindow("e1", Instant.ofEpochMilli(1))); assertTrue(iter.advance()); assertEquals( - iter.getCurrent(), WindowedValues.timestampedValueInGlobalWindow("e2", new Instant(2))); + iter.getCurrent(), + WindowedValues.timestampedValueInGlobalWindow("e2", Instant.ofEpochMilli(2))); assertFalse(iter.advance()); } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubSinkTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubSinkTest.java index 5327cd172410..f459e14515d9 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubSinkTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubSinkTest.java @@ -86,11 +86,14 @@ private void testWriteWith(String formatFn) throws Exception { Sink.SinkWriter> writer = sink.writer(); assertEquals( - 2, writer.add(WindowedValues.timestampedValueInGlobalWindow("e0", new Instant(0)))); + 2, + writer.add(WindowedValues.timestampedValueInGlobalWindow("e0", Instant.ofEpochMilli(0)))); assertEquals( - 2, writer.add(WindowedValues.timestampedValueInGlobalWindow("e1", new Instant(1)))); + 2, + writer.add(WindowedValues.timestampedValueInGlobalWindow("e1", Instant.ofEpochMilli(1)))); assertEquals( - 2, writer.add(WindowedValues.timestampedValueInGlobalWindow("e2", new Instant(2)))); + 2, + writer.add(WindowedValues.timestampedValueInGlobalWindow("e2", Instant.ofEpochMilli(2)))); writer.close(); assertEquals( @@ -174,10 +177,14 @@ public void testExceptionAfterEncoding() throws Exception { assertThrows( "encode error", CoderException.class, - () -> writer.add(WindowedValues.timestampedValueInGlobalWindow("e0", new Instant(0)))); + () -> + writer.add( + WindowedValues.timestampedValueInGlobalWindow("e0", Instant.ofEpochMilli(0)))); assertThrows( "encode error", CoderException.class, - () -> writer.add(WindowedValues.timestampedValueInGlobalWindow("e0", new Instant(0)))); + () -> + writer.add( + WindowedValues.timestampedValueInGlobalWindow("e0", Instant.ofEpochMilli(0)))); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactoryTest.java index 90071d1597ee..294fe4c24786 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactoryTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactoryTest.java @@ -64,8 +64,8 @@ public void testSingleWindow() throws Exception { verifyReifiedIsInTheSameWindows( WindowedValues.of( KV.of(42, "bizzle"), - new Instant(73), - new IntervalWindow(new Instant(5), new Instant(15)), + Instant.ofEpochMilli(73), + new IntervalWindow(Instant.ofEpochMilli(5), Instant.ofEpochMilli(15)), PaneInfo.NO_FIRING)); } @@ -74,10 +74,10 @@ public void testMultiWindowStaysCompressed() throws Exception { verifyReifiedIsInTheSameWindows( WindowedValues.of( KV.of(42, "bizzle"), - new Instant(73), + Instant.ofEpochMilli(73), ImmutableList.of( - new IntervalWindow(new Instant(5), new Instant(15)), - new IntervalWindow(new Instant(17), new Instant(97))), + new IntervalWindow(Instant.ofEpochMilli(5), Instant.ofEpochMilli(15)), + new IntervalWindow(Instant.ofEpochMilli(17), Instant.ofEpochMilli(97))), PaneInfo.NO_FIRING)); } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ShuffleSinkTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ShuffleSinkTest.java index 025eb7c2fbb1..57575c7fa99a 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ShuffleSinkTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ShuffleSinkTest.java @@ -82,7 +82,7 @@ public class ShuffleSinkTest { KV.of(4, KV.of("in 4d", 1)), KV.of(5, KV.of("in 5", 666))); - private static final Instant timestamp = new Instant(123000); + private static final Instant timestamp = Instant.ofEpochMilli(123000); private static final IntervalWindow window = new IntervalWindow(timestamp, timestamp.plus(Duration.millis(1000))); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java index d11c6c374333..7fa1fd866915 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java @@ -231,11 +231,11 @@ public class StreamingDataflowWorkerTest { private static final Logger LOG = LoggerFactory.getLogger(StreamingDataflowWorkerTest.class); private static final IntervalWindow DEFAULT_WINDOW = - new IntervalWindow(new Instant(1234), Duration.millis(1000)); + new IntervalWindow(Instant.ofEpochMilli(1234), Duration.millis(1000)); private static final IntervalWindow WINDOW_AT_ZERO = - new IntervalWindow(new Instant(0), new Instant(1000)); + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(1000)); private static final IntervalWindow WINDOW_AT_ONE_SECOND = - new IntervalWindow(new Instant(1000), new Instant(2000)); + new IntervalWindow(Instant.ofEpochMilli(1000), Instant.ofEpochMilli(2000)); private static final Coder DEFAULT_WINDOW_CODER = IntervalWindow.getCoder(); private static final Coder> DEFAULT_WINDOW_COLLECTION_CODER = CollectionCoder.of(DEFAULT_WINDOW_CODER); @@ -608,8 +608,8 @@ private Windmill.GetWorkResponse buildSessionInput( PaneInfo.NO_FIRING, intervalWindowBytes( new IntervalWindow( - new Instant(input), - new Instant(input).plus(Duration.millis(10))))))); + Instant.ofEpochMilli(input), + Instant.ofEpochMilli(input).plus(Duration.millis(10))))))); } builder.addMessageBundles(messageBuilder); } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowFnsTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowFnsTest.java index 7136538753db..b9249fe27323 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowFnsTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowFnsTest.java @@ -230,7 +230,7 @@ public void testFixedWindows() throws Exception { output -> outputManager.output(outputTag, output), WindowingStrategy.of(FixedWindows.of(Duration.millis(10)))); - when(mockTimerInternals.currentInputWatermarkTime()).thenReturn(new Instant(0)); + when(mockTimerInternals.currentInputWatermarkTime()).thenReturn(Instant.ofEpochMilli(0)); runner.startBundle(); @@ -241,10 +241,14 @@ public void testFixedWindows() throws Exception { messageBundle.setSourceComputationId(SOURCE_COMPUTATION_ID); Coder valueCoder = StringUtf8Coder.of(); - addElement(messageBundle, Arrays.asList(window(0, 10)), new Instant(1), valueCoder, "v1"); - addElement(messageBundle, Arrays.asList(window(0, 10)), new Instant(2), valueCoder, "v2"); - addElement(messageBundle, Arrays.asList(window(0, 10)), new Instant(0), valueCoder, "v0"); - addElement(messageBundle, Arrays.asList(window(10, 20)), new Instant(13), valueCoder, "v3"); + addElement( + messageBundle, Arrays.asList(window(0, 10)), Instant.ofEpochMilli(1), valueCoder, "v1"); + addElement( + messageBundle, Arrays.asList(window(0, 10)), Instant.ofEpochMilli(2), valueCoder, "v2"); + addElement( + messageBundle, Arrays.asList(window(0, 10)), Instant.ofEpochMilli(0), valueCoder, "v0"); + addElement( + messageBundle, Arrays.asList(window(10, 20)), Instant.ofEpochMilli(13), valueCoder, "v3"); runner.processElement(createValue(workItem1, valueCoder)); @@ -254,10 +258,10 @@ public void testFixedWindows() throws Exception { WorkItem.Builder workItem2 = WorkItem.newBuilder(); workItem2.setKey(ByteString.copyFromUtf8(KEY)); workItem2.setWorkToken(WORK_TOKEN); - addTimer(workItem2, window(0, 10), new Instant(9), Timer.Type.WATERMARK); - addTimer(workItem2, window(10, 20), new Instant(19), Timer.Type.WATERMARK); + addTimer(workItem2, window(0, 10), Instant.ofEpochMilli(9), Timer.Type.WATERMARK); + addTimer(workItem2, window(10, 20), Instant.ofEpochMilli(19), Timer.Type.WATERMARK); - when(mockTimerInternals.currentInputWatermarkTime()).thenReturn(new Instant(20)); + when(mockTimerInternals.currentInputWatermarkTime()).thenReturn(Instant.ofEpochMilli(20)); runner.processElement(createValue(workItem2, valueCoder)); runner.finishBundle(); @@ -289,7 +293,7 @@ public void testFixedWindowsWithDraining() throws Exception { output -> outputManager.output(outputTag, output), WindowingStrategy.of(FixedWindows.of(Duration.millis(10)))); - when(mockTimerInternals.currentInputWatermarkTime()).thenReturn(new Instant(0)); + when(mockTimerInternals.currentInputWatermarkTime()).thenReturn(Instant.ofEpochMilli(0)); runner.startBundle(); @@ -300,10 +304,14 @@ public void testFixedWindowsWithDraining() throws Exception { messageBundle.setSourceComputationId(SOURCE_COMPUTATION_ID); Coder valueCoder = StringUtf8Coder.of(); - addElement(messageBundle, Arrays.asList(window(0, 10)), new Instant(1), valueCoder, "v1"); - addElement(messageBundle, Arrays.asList(window(0, 10)), new Instant(2), valueCoder, "v2"); - addElement(messageBundle, Arrays.asList(window(0, 10)), new Instant(0), valueCoder, "v0"); - addElement(messageBundle, Arrays.asList(window(10, 20)), new Instant(13), valueCoder, "v3"); + addElement( + messageBundle, Arrays.asList(window(0, 10)), Instant.ofEpochMilli(1), valueCoder, "v1"); + addElement( + messageBundle, Arrays.asList(window(0, 10)), Instant.ofEpochMilli(2), valueCoder, "v2"); + addElement( + messageBundle, Arrays.asList(window(0, 10)), Instant.ofEpochMilli(0), valueCoder, "v0"); + addElement( + messageBundle, Arrays.asList(window(10, 20)), Instant.ofEpochMilli(13), valueCoder, "v3"); runner.processElement(createValue(workItem1, valueCoder)); runner.finishBundle(); @@ -314,8 +322,8 @@ public void testFixedWindowsWithDraining() throws Exception { workItem2.setWorkToken(WORK_TOKEN); InputMessageBundle.Builder messageBundle2 = workItem1.addMessageBundlesBuilder(); messageBundle2.setSourceComputationId(SOURCE_COMPUTATION_ID); - addTimer(workItem2, window(0, 10), new Instant(9), Timer.Type.WATERMARK); - when(mockTimerInternals.currentInputWatermarkTime()).thenReturn(new Instant(10)); + addTimer(workItem2, window(0, 10), Instant.ofEpochMilli(9), Timer.Type.WATERMARK); + when(mockTimerInternals.currentInputWatermarkTime()).thenReturn(Instant.ofEpochMilli(10)); runner.processElement(createValue(workItem2, valueCoder)); runner.finishBundle(); @@ -325,9 +333,9 @@ public void testFixedWindowsWithDraining() throws Exception { WorkItem.Builder workItem3 = WorkItem.newBuilder(); workItem3.setKey(ByteString.copyFromUtf8(KEY)); workItem3.setWorkToken(WORK_TOKEN); - addTimer(workItem3, window(10, 20), new Instant(19), Timer.Type.WATERMARK); + addTimer(workItem3, window(10, 20), Instant.ofEpochMilli(19), Timer.Type.WATERMARK); - when(mockTimerInternals.currentInputWatermarkTime()).thenReturn(new Instant(20)); + when(mockTimerInternals.currentInputWatermarkTime()).thenReturn(Instant.ofEpochMilli(20)); runner.processElement(createDrainingValue(workItem3, valueCoder)); runner.finishBundle(); @@ -362,7 +370,7 @@ public void testSlidingWindows() throws Exception { WindowingStrategy.of(SlidingWindows.of(Duration.millis(20)).every(Duration.millis(10))) .withTimestampCombiner(TimestampCombiner.EARLIEST)); - when(mockTimerInternals.currentInputWatermarkTime()).thenReturn(new Instant(5)); + when(mockTimerInternals.currentInputWatermarkTime()).thenReturn(Instant.ofEpochMilli(5)); runner.startBundle(); @@ -376,19 +384,19 @@ public void testSlidingWindows() throws Exception { addElement( messageBundle, Arrays.asList(window(-10, 10), window(0, 20)), - new Instant(5), + Instant.ofEpochMilli(5), valueCoder, "v1"); addElement( messageBundle, Arrays.asList(window(-10, 10), window(0, 20)), - new Instant(2), + Instant.ofEpochMilli(2), valueCoder, "v0"); addElement( messageBundle, Arrays.asList(window(0, 20), window(10, 30)), - new Instant(15), + Instant.ofEpochMilli(15), valueCoder, "v2"); @@ -400,10 +408,10 @@ public void testSlidingWindows() throws Exception { WorkItem.Builder workItem2 = WorkItem.newBuilder(); workItem2.setKey(ByteString.copyFromUtf8(KEY)); workItem2.setWorkToken(WORK_TOKEN); - addTimer(workItem2, window(-10, 10), new Instant(9), Timer.Type.WATERMARK); - addTimer(workItem2, window(0, 20), new Instant(19), Timer.Type.WATERMARK); - addTimer(workItem2, window(10, 30), new Instant(29), Timer.Type.WATERMARK); - when(mockTimerInternals.currentInputWatermarkTime()).thenReturn(new Instant(30)); + addTimer(workItem2, window(-10, 10), Instant.ofEpochMilli(9), Timer.Type.WATERMARK); + addTimer(workItem2, window(0, 20), Instant.ofEpochMilli(19), Timer.Type.WATERMARK); + addTimer(workItem2, window(10, 30), Instant.ofEpochMilli(29), Timer.Type.WATERMARK); + when(mockTimerInternals.currentInputWatermarkTime()).thenReturn(Instant.ofEpochMilli(30)); runner.processElement(createValue(workItem2, valueCoder)); @@ -418,15 +426,15 @@ public void testSlidingWindows() throws Exception { containsInAnyOrder( WindowMatchers.isSingleWindowedValue( isKv(equalTo(KEY), containsInAnyOrder("v0", "v1")), - equalTo(new Instant(2)), + equalTo(Instant.ofEpochMilli(2)), equalTo(window(-10, 10))), WindowMatchers.isSingleWindowedValue( isKv(equalTo(KEY), containsInAnyOrder("v0", "v1", "v2")), - equalTo(new Instant(2)), + equalTo(Instant.ofEpochMilli(2)), equalTo(window(0, 20))), WindowMatchers.isSingleWindowedValue( isKv(equalTo(KEY), containsInAnyOrder("v2")), - equalTo(new Instant(15)), + equalTo(Instant.ofEpochMilli(15)), equalTo(window(10, 30))))); } @@ -448,7 +456,7 @@ public void testSlidingWindowsAndLateData() throws Exception { makeRunnerForGabwFn( output -> outputManager.output(outputTag, output), windowingStrategy, fn); - when(mockTimerInternals.currentInputWatermarkTime()).thenReturn(new Instant(15)); + when(mockTimerInternals.currentInputWatermarkTime()).thenReturn(Instant.ofEpochMilli(15)); runner.startBundle(); @@ -462,19 +470,19 @@ public void testSlidingWindowsAndLateData() throws Exception { addElement( messageBundle, Arrays.asList(window(-10, 10), window(0, 20)), - new Instant(5), + Instant.ofEpochMilli(5), valueCoder, "v1"); addElement( messageBundle, Arrays.asList(window(-10, 10), window(0, 20)), - new Instant(2), + Instant.ofEpochMilli(2), valueCoder, "v0"); addElement( messageBundle, Arrays.asList(window(0, 20), window(10, 30)), - new Instant(15), + Instant.ofEpochMilli(15), valueCoder, "v2"); @@ -486,10 +494,10 @@ public void testSlidingWindowsAndLateData() throws Exception { WorkItem.Builder workItem2 = WorkItem.newBuilder(); workItem2.setKey(ByteString.copyFromUtf8(KEY)); workItem2.setWorkToken(WORK_TOKEN); - addTimer(workItem2, window(-10, 10), new Instant(9), Timer.Type.WATERMARK); - addTimer(workItem2, window(0, 20), new Instant(19), Timer.Type.WATERMARK); - addTimer(workItem2, window(10, 30), new Instant(29), Timer.Type.WATERMARK); - when(mockTimerInternals.currentInputWatermarkTime()).thenReturn(new Instant(30)); + addTimer(workItem2, window(-10, 10), Instant.ofEpochMilli(9), Timer.Type.WATERMARK); + addTimer(workItem2, window(0, 20), Instant.ofEpochMilli(19), Timer.Type.WATERMARK); + addTimer(workItem2, window(10, 30), Instant.ofEpochMilli(29), Timer.Type.WATERMARK); + when(mockTimerInternals.currentInputWatermarkTime()).thenReturn(Instant.ofEpochMilli(30)); runner.processElement(createValue(workItem2, valueCoder)); @@ -508,11 +516,11 @@ public void testSlidingWindowsAndLateData() throws Exception { equalTo(window(-10, 10))), WindowMatchers.isSingleWindowedValue( isKv(equalTo(KEY), containsInAnyOrder("v0", "v1", "v2")), - equalTo(new Instant(2)), + equalTo(Instant.ofEpochMilli(2)), equalTo(window(0, 20))), WindowMatchers.isSingleWindowedValue( isKv(equalTo(KEY), containsInAnyOrder("v2")), - equalTo(new Instant(15)), + equalTo(Instant.ofEpochMilli(15)), equalTo(window(10, 30))))); long droppedValues = @@ -536,7 +544,7 @@ public void testSessions() throws Exception { WindowingStrategy.of(Sessions.withGapDuration(Duration.millis(10))) .withTimestampCombiner(TimestampCombiner.EARLIEST)); - when(mockTimerInternals.currentInputWatermarkTime()).thenReturn(new Instant(0)); + when(mockTimerInternals.currentInputWatermarkTime()).thenReturn(Instant.ofEpochMilli(0)); runner.startBundle(); @@ -547,11 +555,15 @@ public void testSessions() throws Exception { messageBundle.setSourceComputationId(SOURCE_COMPUTATION_ID); Coder valueCoder = StringUtf8Coder.of(); - addElement(messageBundle, Arrays.asList(window(0, 10)), new Instant(0), valueCoder, "v1"); - addElement(messageBundle, Arrays.asList(window(5, 15)), new Instant(5), valueCoder, "v2"); - addElement(messageBundle, Arrays.asList(window(15, 25)), new Instant(15), valueCoder, "v3"); + addElement( + messageBundle, Arrays.asList(window(0, 10)), Instant.ofEpochMilli(0), valueCoder, "v1"); + addElement( + messageBundle, Arrays.asList(window(5, 15)), Instant.ofEpochMilli(5), valueCoder, "v2"); + addElement( + messageBundle, Arrays.asList(window(15, 25)), Instant.ofEpochMilli(15), valueCoder, "v3"); - addElement(messageBundle, Arrays.asList(window(3, 13)), new Instant(3), valueCoder, "v0"); + addElement( + messageBundle, Arrays.asList(window(3, 13)), Instant.ofEpochMilli(3), valueCoder, "v0"); runner.processElement(createValue(workItem1, valueCoder)); @@ -563,9 +575,9 @@ public void testSessions() throws Exception { workItem2.setWorkToken(WORK_TOKEN); // Note that the WATERMARK timer for Instant(9) will have been deleted by // ReduceFnRunner when window(0, 10) was merged away. - addTimer(workItem2, window(0, 15), new Instant(14), Timer.Type.WATERMARK); - addTimer(workItem2, window(15, 25), new Instant(24), Timer.Type.WATERMARK); - when(mockTimerInternals.currentInputWatermarkTime()).thenReturn(new Instant(25)); + addTimer(workItem2, window(0, 15), Instant.ofEpochMilli(14), Timer.Type.WATERMARK); + addTimer(workItem2, window(15, 25), Instant.ofEpochMilli(24), Timer.Type.WATERMARK); + when(mockTimerInternals.currentInputWatermarkTime()).thenReturn(Instant.ofEpochMilli(25)); runner.processElement(createValue(workItem2, valueCoder)); @@ -580,11 +592,11 @@ public void testSessions() throws Exception { containsInAnyOrder( WindowMatchers.isSingleWindowedValue( isKv(equalTo(KEY), containsInAnyOrder("v0", "v1", "v2")), - equalTo(new Instant(0)), + equalTo(Instant.ofEpochMilli(0)), equalTo(window(0, 15))), WindowMatchers.isSingleWindowedValue( isKv(equalTo(KEY), containsInAnyOrder("v3")), - equalTo(new Instant(15)), + equalTo(Instant.ofEpochMilli(15)), equalTo(window(15, 25))))); } @@ -653,7 +665,7 @@ public void testSessionsCombine() throws Exception { WindowingStrategy.of(Sessions.withGapDuration(Duration.millis(10))), appliedCombineFn); - when(mockTimerInternals.currentInputWatermarkTime()).thenReturn(new Instant(0)); + when(mockTimerInternals.currentInputWatermarkTime()).thenReturn(Instant.ofEpochMilli(0)); runner.startBundle(); @@ -664,10 +676,14 @@ public void testSessionsCombine() throws Exception { messageBundle.setSourceComputationId(SOURCE_COMPUTATION_ID); Coder valueCoder = BigEndianLongCoder.of(); - addElement(messageBundle, Arrays.asList(window(0, 10)), new Instant(0), valueCoder, 1L); - addElement(messageBundle, Arrays.asList(window(5, 15)), new Instant(5), valueCoder, 2L); - addElement(messageBundle, Arrays.asList(window(15, 25)), new Instant(15), valueCoder, 3L); - addElement(messageBundle, Arrays.asList(window(3, 13)), new Instant(3), valueCoder, 4L); + addElement( + messageBundle, Arrays.asList(window(0, 10)), Instant.ofEpochMilli(0), valueCoder, 1L); + addElement( + messageBundle, Arrays.asList(window(5, 15)), Instant.ofEpochMilli(5), valueCoder, 2L); + addElement( + messageBundle, Arrays.asList(window(15, 25)), Instant.ofEpochMilli(15), valueCoder, 3L); + addElement( + messageBundle, Arrays.asList(window(3, 13)), Instant.ofEpochMilli(3), valueCoder, 4L); runner.processElement(createValue(workItem1, valueCoder)); @@ -679,9 +695,9 @@ public void testSessionsCombine() throws Exception { workItem2.setWorkToken(WORK_TOKEN); // Note that the WATERMARK timer for Instant(9) will have been deleted by // ReduceFnRunner when window(0, 10) was merged away. - addTimer(workItem2, window(0, 15), new Instant(14), Timer.Type.WATERMARK); - addTimer(workItem2, window(15, 25), new Instant(24), Timer.Type.WATERMARK); - when(mockTimerInternals.currentInputWatermarkTime()).thenReturn(new Instant(25)); + addTimer(workItem2, window(0, 15), Instant.ofEpochMilli(14), Timer.Type.WATERMARK); + addTimer(workItem2, window(15, 25), Instant.ofEpochMilli(24), Timer.Type.WATERMARK); + when(mockTimerInternals.currentInputWatermarkTime()).thenReturn(Instant.ofEpochMilli(25)); runner.processElement(createValue(workItem2, valueCoder)); @@ -752,6 +768,6 @@ DoFnRunner, KV> makeRunnerForGabw } private IntervalWindow window(long start, long end) { - return new IntervalWindow(new Instant(start), new Instant(end)); + return new IntervalWindow(Instant.ofEpochMilli(start), Instant.ofEpochMilli(end)); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowsReshuffleDoFnTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowsReshuffleDoFnTest.java index bdeefcebb2ac..b65622d812fd 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowsReshuffleDoFnTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowsReshuffleDoFnTest.java @@ -158,10 +158,14 @@ public void testFixedWindows() throws Exception { messageBundle.setSourceComputationId(SOURCE_COMPUTATION_ID); Coder valueCoder = StringUtf8Coder.of(); - addElement(messageBundle, Arrays.asList(window(0, 10)), new Instant(1), valueCoder, "v1"); - addElement(messageBundle, Arrays.asList(window(0, 10)), new Instant(2), valueCoder, "v2"); - addElement(messageBundle, Arrays.asList(window(0, 10)), new Instant(0), valueCoder, "v0"); - addElement(messageBundle, Arrays.asList(window(10, 20)), new Instant(13), valueCoder, "v3"); + addElement( + messageBundle, Arrays.asList(window(0, 10)), Instant.ofEpochMilli(1), valueCoder, "v1"); + addElement( + messageBundle, Arrays.asList(window(0, 10)), Instant.ofEpochMilli(2), valueCoder, "v2"); + addElement( + messageBundle, Arrays.asList(window(0, 10)), Instant.ofEpochMilli(0), valueCoder, "v0"); + addElement( + messageBundle, Arrays.asList(window(10, 20)), Instant.ofEpochMilli(13), valueCoder, "v3"); runner.processElement(createValue(workItem, valueCoder)); @@ -174,25 +178,25 @@ public void testFixedWindows() throws Exception { WindowedValue>> item0 = result.get(0); assertEquals(KEY, item0.getValue().getKey()); assertThat(item0.getValue().getValue(), Matchers.containsInAnyOrder("v1")); - assertEquals(new Instant(1), item0.getTimestamp()); + assertEquals(Instant.ofEpochMilli(1), item0.getTimestamp()); assertThat(item0.getWindows(), Matchers.contains(window(0, 10))); WindowedValue>> item1 = result.get(1); assertEquals(KEY, item1.getValue().getKey()); assertThat(item1.getValue().getValue(), Matchers.containsInAnyOrder("v2")); - assertEquals(new Instant(2), item1.getTimestamp()); + assertEquals(Instant.ofEpochMilli(2), item1.getTimestamp()); assertThat(item1.getWindows(), Matchers.contains(window(0, 10))); WindowedValue>> item2 = result.get(2); assertEquals(KEY, item2.getValue().getKey()); assertThat(item2.getValue().getValue(), Matchers.containsInAnyOrder("v0")); - assertEquals(new Instant(0), item2.getTimestamp()); + assertEquals(Instant.ofEpochMilli(0), item2.getTimestamp()); assertThat(item2.getWindows(), Matchers.contains(window(0, 10))); WindowedValue>> item3 = result.get(3); assertEquals(KEY, item3.getValue().getKey()); assertThat(item3.getValue().getValue(), Matchers.containsInAnyOrder("v3")); - assertEquals(new Instant(13), item3.getTimestamp()); + assertEquals(Instant.ofEpochMilli(13), item3.getTimestamp()); assertThat(item3.getWindows(), Matchers.contains(window(10, 20))); } @@ -219,6 +223,6 @@ DoFnRunner, KV> makeRunner( } private IntervalWindow window(long start, long end) { - return new IntervalWindow(new Instant(start), new Instant(end)); + return new IntervalWindow(Instant.ofEpochMilli(start), Instant.ofEpochMilli(end)); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunnerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunnerTest.java index e12ddd95f913..86d1c8cc575b 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunnerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunnerTest.java @@ -99,7 +99,7 @@ public void testInvokeProcessElement() throws Exception { when(sideInputFetcher.storeIfBlocked(ArgumentMatchers.>any())) .thenReturn(false, true, false) .thenThrow(new RuntimeException("Does not expect more calls")); - when(mockTimerInternals.currentInputWatermarkTime()).thenReturn(new Instant(15L)); + when(mockTimerInternals.currentInputWatermarkTime()).thenReturn(Instant.ofEpochMilli(15L)); ListOutputManager outputManager = new ListOutputManager(); StreamingKeyedWorkItemSideInputDoFnRunner, IntervalWindow> runner = createRunner(outputManager); @@ -114,13 +114,13 @@ public void testInvokeProcessElement() throws Exception { runner.processElement(new ValueInEmptyWindows<>(elemsWorkItem)); - when(mockTimerInternals.currentInputWatermarkTime()).thenReturn(new Instant(20)); + when(mockTimerInternals.currentInputWatermarkTime()).thenReturn(Instant.ofEpochMilli(20)); runner.processElement( new ValueInEmptyWindows<>( KeyedWorkItems.timersWorkItem( "a", ImmutableList.of( - timerData(window(10, 20), new Instant(19), Timer.Type.WATERMARK))))); + timerData(window(10, 20), Instant.ofEpochMilli(19), Timer.Type.WATERMARK))))); List>> result = outputManager.getOutput(mainOutputTag); assertEquals(1, result.size()); @@ -144,8 +144,9 @@ public void testStartBundle() throws Exception { when(elemsBag.read()).thenReturn(ImmutableList.of(createDatum(13, 13L), createDatum(18, 18L))); when(timersBag.read()) .thenReturn( - ImmutableList.of(timerData(window(10, 20), new Instant(19), Timer.Type.WATERMARK))); - when(mockTimerInternals.currentInputWatermarkTime()).thenReturn(new Instant(20)); + ImmutableList.of( + timerData(window(10, 20), Instant.ofEpochMilli(19), Timer.Type.WATERMARK))); + when(mockTimerInternals.currentInputWatermarkTime()).thenReturn(Instant.ofEpochMilli(20)); runner.startBundle(); @@ -157,7 +158,7 @@ public void testStartBundle() throws Exception { } private WindowedValue createDatum(T element, long timestampMillis) { - Instant timestamp = new Instant(timestampMillis); + Instant timestamp = Instant.ofEpochMilli(timestampMillis); return WindowedValues.of( element, timestamp, Arrays.asList(WINDOW_FN.assignWindow(timestamp)), PaneInfo.NO_FIRING); } @@ -172,7 +173,7 @@ private TimerData timerData(IntervalWindow window, Instant timestamp, Timer.Type } private IntervalWindow window(long start, long end) { - return new IntervalWindow(new Instant(start), new Instant(end)); + return new IntervalWindow(Instant.ofEpochMilli(start), Instant.ofEpochMilli(end)); } @SuppressWarnings("unchecked") diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java index 4bfa6efc8880..db48222c5f81 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java @@ -165,7 +165,7 @@ public void testTimerInternalsSetTimer() { "key", createMockWork( Windmill.WorkItem.newBuilder().setKey(ByteString.EMPTY).setWorkToken(17L).build(), - Watermarks.builder().setInputDataWatermark(new Instant(1000)).build()), + Watermarks.builder().setInputDataWatermark(Instant.ofEpochMilli(1000)).build()), stateReader, sideInputStateFetcher, outputBuilder); @@ -175,8 +175,8 @@ public void testTimerInternalsSetTimer() { timerInternals.setTimer( TimerData.of( new StateNamespaceForTest("key"), - new Instant(5000), - new Instant(5000), + Instant.ofEpochMilli(5000), + Instant.ofEpochMilli(5000), TimeDomain.EVENT_TIME, CausedByDrain.NORMAL)); executionContext.flushState(); @@ -215,7 +215,7 @@ public void testTimerInternalsProcessingTimeSkew() { "key", createMockWork( workItemBuilder.build(), - Watermarks.builder().setInputDataWatermark(new Instant(1000)).build()), + Watermarks.builder().setInputDataWatermark(Instant.ofEpochMilli(1000)).build()), stateReader, sideInputStateFetcher, outputBuilder); @@ -424,7 +424,7 @@ public void testStateTagEncodingBasedOnConfig() { "key", createMockWork( Windmill.WorkItem.newBuilder().setKey(ByteString.EMPTY).setWorkToken(17L).build(), - Watermarks.builder().setInputDataWatermark(new Instant(1000)).build()), + Watermarks.builder().setInputDataWatermark(Instant.ofEpochMilli(1000)).build()), stateReader, sideInputStateFetcher, outputBuilder); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunnerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunnerTest.java index d18bc512723e..81be04ce0505 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunnerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunnerTest.java @@ -154,7 +154,7 @@ public void testSideInputNotReady() throws Exception { assertTrue(outputManager.getOutput(mainOutputTag).isEmpty()); - IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(10)); + IntervalWindow window = new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10)); // Verify that we added the element to an appropriate tag list, and that we buffered the element ValueState>> blockedMapState = @@ -180,7 +180,7 @@ public void testSideInputNotReady() throws Exception { assertThat( sideInputFetcher.elementBag(createWindow(0)).read(), Matchers.contains(createDatum("e", 0))); - assertEquals(sideInputFetcher.watermarkHold(createWindow(0)).read(), new Instant(0)); + assertEquals(sideInputFetcher.watermarkHold(createWindow(0)).read(), Instant.ofEpochMilli(0)); } @Test @@ -204,13 +204,16 @@ public void testMultipleWindowsNotReady() throws Exception { views, sideInputFetcher); - IntervalWindow window1 = new IntervalWindow(new Instant(0), new Instant(10)); - IntervalWindow window2 = new IntervalWindow(new Instant(-5), new Instant(5)); + IntervalWindow window1 = new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10)); + IntervalWindow window2 = new IntervalWindow(Instant.ofEpochMilli(-5), Instant.ofEpochMilli(5)); long timestamp = 1L; WindowedValue elem = WindowedValues.of( - "e", new Instant(timestamp), Arrays.asList(window1, window2), PaneInfo.NO_FIRING); + "e", + Instant.ofEpochMilli(timestamp), + Arrays.asList(window1, window2), + PaneInfo.NO_FIRING); runner.startBundle(); runner.processElement(elem); @@ -267,15 +270,15 @@ public void testMultipleWindowsNotReady() throws Exception { sideInputFetcher.elementBag(window2).read(), contains(Iterables.get(elem.explodeWindows(), 1))); - assertEquals(sideInputFetcher.watermarkHold(window1).read(), new Instant(timestamp)); - assertEquals(sideInputFetcher.watermarkHold(window2).read(), new Instant(timestamp)); + assertEquals(sideInputFetcher.watermarkHold(window1).read(), Instant.ofEpochMilli(timestamp)); + assertEquals(sideInputFetcher.watermarkHold(window2).read(), Instant.ofEpochMilli(timestamp)); } @Test public void testSideInputNotification() throws Exception { PCollectionView view = createView(); - IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(10)); + IntervalWindow window = new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10)); Windmill.GlobalDataId id = Windmill.GlobalDataId.newBuilder() .setTag(view.getTagInternal().getId()) @@ -300,7 +303,7 @@ public void testSideInputNotification() throws Exception { StreamingSideInputFetcher sideInputFetcher = createFetcher(views); StreamingSideInputDoFnRunner runner = createRunner(outputManager, views, sideInputFetcher); - sideInputFetcher.watermarkHold(createWindow(0)).add(new Instant(0)); + sideInputFetcher.watermarkHold(createWindow(0)).add(Instant.ofEpochMilli(0)); sideInputFetcher.elementBag(createWindow(0)).add(createDatum("e", 0)); when(stepContext.getSideInputNotifications()).thenReturn(Arrays.asList(id)); @@ -331,7 +334,7 @@ public void testMultipleSideInputs() throws Exception { PCollectionView view1 = createView(); PCollectionView view2 = createView(); - IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(10)); + IntervalWindow window = new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10)); Windmill.GlobalDataId id = Windmill.GlobalDataId.newBuilder() .setTag(view1.getTagInternal().getId()) @@ -368,7 +371,7 @@ public void testMultipleSideInputs() throws Exception { StreamingSideInputFetcher sideInputFetcher = createFetcher(views); StreamingSideInputDoFnRunner runner = createRunner(outputManager, views, sideInputFetcher); - sideInputFetcher.watermarkHold(createWindow(0)).add(new Instant(0)); + sideInputFetcher.watermarkHold(createWindow(0)).add(Instant.ofEpochMilli(0)); sideInputFetcher.elementBag(createWindow(0)).add(createDatum("e1", 0)); runner.startBundle(); @@ -451,13 +454,14 @@ private PCollectionView createView() { private WindowedValue createDatum(String element, long timestamp) { return WindowedValues.of( element, - new Instant(timestamp), + Instant.ofEpochMilli(timestamp), Arrays.asList(createWindow(timestamp)), PaneInfo.NO_FIRING); } private IntervalWindow createWindow(long timestamp) { return new IntervalWindow( - new Instant(timestamp - timestamp % 10), new Instant(timestamp - timestamp % 10 + 10)); + Instant.ofEpochMilli(timestamp - timestamp % 10), + Instant.ofEpochMilli(timestamp - timestamp % 10 + 10)); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcherTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcherTest.java index 06db68ccc324..f26e32b896c6 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcherTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcherTest.java @@ -201,7 +201,7 @@ private PCollectionView createView() { private WindowedValue createDatum(String element, long timestamp) { return WindowedValues.of( element, - new Instant(timestamp), + Instant.ofEpochMilli(timestamp), Arrays.asList(createWindow(timestamp)), PaneInfo.NO_FIRING); } @@ -209,14 +209,15 @@ private WindowedValue createDatum(String element, long timestamp) { private TimerData createTimer(long timestamp) { return TimerData.of( StateNamespaces.window(IntervalWindow.getCoder(), createWindow(timestamp)), - new Instant(timestamp), - new Instant(timestamp), + Instant.ofEpochMilli(timestamp), + Instant.ofEpochMilli(timestamp), TimeDomain.EVENT_TIME, CausedByDrain.NORMAL); } private IntervalWindow createWindow(long timestamp) { return new IntervalWindow( - new Instant(timestamp - timestamp % 10), new Instant(timestamp - timestamp % 10 + 10)); + Instant.ofEpochMilli(timestamp - timestamp % 10), + Instant.ofEpochMilli(timestamp - timestamp % 10 + 10)); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/UngroupedShuffleReaderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/UngroupedShuffleReaderTest.java index 5e8fe648e7d7..33f7f9bea86a 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/UngroupedShuffleReaderTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/UngroupedShuffleReaderTest.java @@ -43,7 +43,7 @@ /** Tests for UngroupedShuffleReader. */ @RunWith(JUnit4.class) public class UngroupedShuffleReaderTest { - private static final Instant timestamp = new Instant(123000); + private static final Instant timestamp = Instant.ofEpochMilli(123000); private static final IntervalWindow window = new IntervalWindow(timestamp, timestamp.minus(Duration.millis(1000))); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/UserParDoFnFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/UserParDoFnFactoryTest.java index 9d3fa9b211b1..16b324c2247a 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/UserParDoFnFactoryTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/UserParDoFnFactoryTest.java @@ -377,9 +377,10 @@ public void testCleanupRegistered() throws Exception { Receiver rcvr = new OutputReceiver(); parDoFn.startBundle(rcvr); - IntervalWindow firstWindow = new IntervalWindow(new Instant(0), new Instant(10)); + IntervalWindow firstWindow = + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10)); parDoFn.processElement( - WindowedValues.of("foo", new Instant(1), firstWindow, PaneInfo.NO_FIRING)); + WindowedValues.of("foo", Instant.ofEpochMilli(1), firstWindow, PaneInfo.NO_FIRING)); verify(stepContext) .setStateCleanupTimer( @@ -436,7 +437,7 @@ public void testCleanupTimerForGlobalWindowWithAllowedLateness() throws Exceptio GlobalWindow globalWindow = GlobalWindow.INSTANCE; parDoFn.processElement( - WindowedValues.of("foo", new Instant(1), globalWindow, PaneInfo.NO_FIRING)); + WindowedValues.of("foo", Instant.ofEpochMilli(1), globalWindow, PaneInfo.NO_FIRING)); assertThat( globalWindow.maxTimestamp().plus(allowedLateness), @@ -516,8 +517,10 @@ public void testCleanupWorks() throws Exception { Receiver rcvr = new OutputReceiver(); parDoFn.startBundle(rcvr); - IntervalWindow firstWindow = new IntervalWindow(new Instant(0), new Instant(9)); - IntervalWindow secondWindow = new IntervalWindow(new Instant(10), new Instant(19)); + IntervalWindow firstWindow = + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(9)); + IntervalWindow secondWindow = + new IntervalWindow(Instant.ofEpochMilli(10), Instant.ofEpochMilli(19)); Coder windowCoder = IntervalWindow.getCoder(); StateNamespace firstWindowNamespace = StateNamespaces.window(windowCoder, firstWindow); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillKeyedWorkItemTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillKeyedWorkItemTest.java index 69aa4d0d69af..ccda49c7dafe 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillKeyedWorkItemTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillKeyedWorkItemTest.java @@ -72,11 +72,11 @@ public class WindmillKeyedWorkItemTest { private static final Coder VALUE_CODER = StringUtf8Coder.of(); private static final IntervalWindow WINDOW_1 = - new IntervalWindow(new Instant(0), new Instant(10)); + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10)); private static final StateNamespace STATE_NAMESPACE_1 = StateNamespaces.window(WINDOW_CODER, WINDOW_1); private static final IntervalWindow WINDOW_2 = - new IntervalWindow(new Instant(10), new Instant(20)); + new IntervalWindow(Instant.ofEpochMilli(10), Instant.ofEpochMilli(20)); private static final StateNamespace STATE_NAMESPACE_2 = StateNamespaces.window(WINDOW_CODER, WINDOW_2); @@ -113,9 +113,9 @@ public void testElementIteration() throws Exception { assertThat( keyedWorkItem.elementsIterable(), Matchers.contains( - WindowedValues.of("hello", new Instant(5), WINDOW_1, paneInfo(0)), - WindowedValues.of("world", new Instant(7), WINDOW_2, paneInfo(2)), - WindowedValues.of("earth", new Instant(6), WINDOW_1, paneInfo(1)))); + WindowedValues.of("hello", Instant.ofEpochMilli(5), WINDOW_1, paneInfo(0)), + WindowedValues.of("world", Instant.ofEpochMilli(7), WINDOW_2, paneInfo(2)), + WindowedValues.of("earth", Instant.ofEpochMilli(6), WINDOW_1, paneInfo(1)))); } private void addElement( @@ -133,7 +133,7 @@ private void addElement( BeamFnApi.Elements.ElementMetadata.newBuilder().build()); chunk .addMessagesBuilder() - .setTimestamp(WindmillTimeUtils.harnessToWindmillTimestamp(new Instant(timestamp))) + .setTimestamp(WindmillTimeUtils.harnessToWindmillTimestamp(Instant.ofEpochMilli(timestamp))) .setData(ByteString.copyFromUtf8(value)) .setMetadata(encodedMetadata); } @@ -151,7 +151,7 @@ private void addElementWithMetadata( WINDOWS_CODER, Collections.singletonList(window), pane, metadata); chunk .addMessagesBuilder() - .setTimestamp(WindmillTimeUtils.harnessToWindmillTimestamp(new Instant(timestamp))) + .setTimestamp(WindmillTimeUtils.harnessToWindmillTimestamp(Instant.ofEpochMilli(timestamp))) .setData(ByteString.copyFromUtf8(value)) .setMetadata(encodedMetadata); } @@ -201,11 +201,11 @@ private Windmill.Timer makeSerializedTimer( WindmillTimerType.SYSTEM_TIMER, TimerData.of( ns, - new Instant(timestamp), - new Instant(timestamp), + Instant.ofEpochMilli(timestamp), + Instant.ofEpochMilli(timestamp), timerTypeToTimeDomain(type), CausedByDrain.NORMAL))) - .setTimestamp(WindmillTimeUtils.harnessToWindmillTimestamp(new Instant(timestamp))) + .setTimestamp(WindmillTimeUtils.harnessToWindmillTimestamp(Instant.ofEpochMilli(timestamp))) .setType(type) .setStateFamily(STATE_FAMILY) .build(); @@ -213,12 +213,20 @@ private Windmill.Timer makeSerializedTimer( private static TimerData makeTimer(StateNamespace ns, long timestamp, TimeDomain domain) { return TimerData.of( - ns, new Instant(timestamp), new Instant(timestamp), domain, CausedByDrain.NORMAL); + ns, + Instant.ofEpochMilli(timestamp), + Instant.ofEpochMilli(timestamp), + domain, + CausedByDrain.NORMAL); } private static TimerData makeDrainingTimer(StateNamespace ns, long timestamp, TimeDomain domain) { return TimerData.of( - ns, new Instant(timestamp), new Instant(timestamp), domain, CausedByDrain.CAUSED_BY_DRAIN); + ns, + Instant.ofEpochMilli(timestamp), + Instant.ofEpochMilli(timestamp), + domain, + CausedByDrain.CAUSED_BY_DRAIN); } @Test diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillTimeUtilsTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillTimeUtilsTest.java index 84e76d2f8bd1..e266726c842d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillTimeUtilsTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillTimeUtilsTest.java @@ -38,12 +38,12 @@ public void testWindmillToHarnessWatermark() { assertEquals(BoundedWindow.TIMESTAMP_MAX_VALUE, windmillToHarnessWatermark(Long.MAX_VALUE)); assertEquals( BoundedWindow.TIMESTAMP_MAX_VALUE, windmillToHarnessWatermark(Long.MAX_VALUE - 17)); - assertEquals(new Instant(16), windmillToHarnessWatermark(16999)); - assertEquals(new Instant(17), windmillToHarnessWatermark(17120)); - assertEquals(new Instant(17), windmillToHarnessWatermark(17000)); - assertEquals(new Instant(-17), windmillToHarnessWatermark(-16987)); - assertEquals(new Instant(-17), windmillToHarnessWatermark(-17000)); - assertEquals(new Instant(-18), windmillToHarnessTimestamp(-17001)); + assertEquals(Instant.ofEpochMilli(16), windmillToHarnessWatermark(16999)); + assertEquals(Instant.ofEpochMilli(17), windmillToHarnessWatermark(17120)); + assertEquals(Instant.ofEpochMilli(17), windmillToHarnessWatermark(17000)); + assertEquals(Instant.ofEpochMilli(-17), windmillToHarnessWatermark(-16987)); + assertEquals(Instant.ofEpochMilli(-17), windmillToHarnessWatermark(-17000)); + assertEquals(Instant.ofEpochMilli(-18), windmillToHarnessTimestamp(-17001)); } @Test @@ -51,12 +51,12 @@ public void testWindmillToHarnessTimestamp() { assertEquals(BoundedWindow.TIMESTAMP_MAX_VALUE, windmillToHarnessTimestamp(Long.MAX_VALUE)); assertEquals( BoundedWindow.TIMESTAMP_MAX_VALUE, windmillToHarnessTimestamp(Long.MAX_VALUE - 17)); - assertEquals(new Instant(16), windmillToHarnessWatermark(16999)); - assertEquals(new Instant(17), windmillToHarnessTimestamp(17120)); - assertEquals(new Instant(17), windmillToHarnessTimestamp(17000)); - assertEquals(new Instant(-17), windmillToHarnessTimestamp(-16987)); - assertEquals(new Instant(-17), windmillToHarnessTimestamp(-17000)); - assertEquals(new Instant(-18), windmillToHarnessTimestamp(-17001)); + assertEquals(Instant.ofEpochMilli(16), windmillToHarnessWatermark(16999)); + assertEquals(Instant.ofEpochMilli(17), windmillToHarnessTimestamp(17120)); + assertEquals(Instant.ofEpochMilli(17), windmillToHarnessTimestamp(17000)); + assertEquals(Instant.ofEpochMilli(-17), windmillToHarnessTimestamp(-16987)); + assertEquals(Instant.ofEpochMilli(-17), windmillToHarnessTimestamp(-17000)); + assertEquals(Instant.ofEpochMilli(-18), windmillToHarnessTimestamp(-17001)); assertEquals(BoundedWindow.TIMESTAMP_MIN_VALUE, windmillToHarnessTimestamp(Long.MIN_VALUE + 1)); assertEquals(BoundedWindow.TIMESTAMP_MIN_VALUE, windmillToHarnessTimestamp(Long.MIN_VALUE + 2)); // Long.MIN_VALUE = -9223372036854775808, need to add 1808 microseconds to get to next @@ -71,10 +71,12 @@ public void testWindmillToHarnessTimestamp() { @Test public void testHarnessToWindmillTimestamp() { assertEquals(Long.MAX_VALUE, harnessToWindmillTimestamp(BoundedWindow.TIMESTAMP_MAX_VALUE)); - assertEquals(-1000, harnessToWindmillTimestamp(new Instant(-1))); - assertEquals(1000, harnessToWindmillTimestamp(new Instant(1))); - assertEquals(Long.MIN_VALUE + 1, harnessToWindmillTimestamp(new Instant(Long.MIN_VALUE))); + assertEquals(-1000, harnessToWindmillTimestamp(Instant.ofEpochMilli(-1))); + assertEquals(1000, harnessToWindmillTimestamp(Instant.ofEpochMilli(1))); assertEquals( - Long.MIN_VALUE + 1, harnessToWindmillTimestamp(new Instant(Long.MIN_VALUE / 1000 - 1))); + Long.MIN_VALUE + 1, harnessToWindmillTimestamp(Instant.ofEpochMilli(Long.MIN_VALUE))); + assertEquals( + Long.MIN_VALUE + 1, + harnessToWindmillTimestamp(Instant.ofEpochMilli(Long.MIN_VALUE / 1000 - 1))); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java index 334b9414b26b..7e3094d04cc7 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java @@ -639,7 +639,7 @@ public void testReadUnboundedReader() throws Exception { .setSourceState( Windmill.SourceState.newBuilder().setState(state).build()) // Source state. .build(), - Watermarks.builder().setInputDataWatermark(new Instant(0)).build()), + Watermarks.builder().setInputDataWatermark(Instant.ofEpochMilli(0)).build()), mock(WindmillStateReader.class), mock(SideInputStateFetcher.class), Windmill.WorkItemCommitRequest.newBuilder()); @@ -1009,7 +1009,7 @@ public void testFailedWorkItemsAbort() throws Exception { Work.create( workItem, workItem.getSerializedSize(), - Watermarks.builder().setInputDataWatermark(new Instant(0)).build(), + Watermarks.builder().setInputDataWatermark(Instant.ofEpochMilli(0)).build(), Work.createProcessingContext( COMPUTATION_ID, new FakeGetDataClient(), diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/testing/TestCountingSource.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/testing/TestCountingSource.java index 21e4d8c55e70..27c98aaee41c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/testing/TestCountingSource.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/testing/TestCountingSource.java @@ -182,7 +182,7 @@ public KV getCurrent() { @Override public Instant getCurrentTimestamp() { - return new Instant(current); + return Instant.ofEpochMilli(current); } @Override @@ -206,7 +206,7 @@ public TestCountingSource getCurrentSource() { public Instant getWatermark() { // The watermark is a promise about future elements, and the timestamps of elements are // strictly increasing for this source. - return new Instant(current + 1); + return Instant.ofEpochMilli(current + 1); } @Override diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowReshuffleDoFnTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowReshuffleDoFnTest.java index 79e2aee7fedd..6dd4f3f27679 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowReshuffleDoFnTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowReshuffleDoFnTest.java @@ -134,7 +134,7 @@ DoFnRunner>>, KV> makeRunner( } private static BoundedWindow window(long start, long end) { - return new IntervalWindow(new Instant(start), new Instant(end)); + return new IntervalWindow(Instant.ofEpochMilli(start), Instant.ofEpochMilli(end)); } @Test @@ -159,27 +159,27 @@ public void testReshuffleFiresEveryElement() throws Exception { windowingStrategy, "key", WindowedValues.of( - "v1", new Instant(1), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), + "v1", Instant.ofEpochMilli(1), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), WindowedValues.of( - "v2", new Instant(2), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), + "v2", Instant.ofEpochMilli(2), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), WindowedValues.of( - "v3", new Instant(13), Arrays.asList(window(10, 20)), PaneInfo.NO_FIRING)); + "v3", Instant.ofEpochMilli(13), Arrays.asList(window(10, 20)), PaneInfo.NO_FIRING)); assertThat(result.size(), equalTo(3)); WindowedValue>> item0 = result.get(0); assertThat(item0.getValue().getValue(), contains("v1")); - assertThat(item0.getTimestamp(), equalTo(new Instant(1))); + assertThat(item0.getTimestamp(), equalTo(Instant.ofEpochMilli(1))); assertThat(item0.getWindows(), contains(window(0, 10))); WindowedValue>> item1 = result.get(1); assertThat(item1.getValue().getValue(), contains("v2")); - assertThat(item1.getTimestamp(), equalTo(new Instant(2))); + assertThat(item1.getTimestamp(), equalTo(Instant.ofEpochMilli(2))); assertThat(item1.getWindows(), contains(window(0, 10))); WindowedValue>> item2 = result.get(2); assertThat(item2.getValue().getValue(), contains("v3")); - assertThat(item2.getTimestamp(), equalTo(new Instant(13))); + assertThat(item2.getTimestamp(), equalTo(Instant.ofEpochMilli(13))); assertThat(item2.getWindows(), contains(window(10, 20))); } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/GroupAlsoByWindowProperties.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/GroupAlsoByWindowProperties.java index aec6b474e7d5..08399e5fccc3 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/GroupAlsoByWindowProperties.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/GroupAlsoByWindowProperties.java @@ -117,11 +117,11 @@ public static void groupsElementsIntoFixedWindows( windowingStrategy, "key", WindowedValues.of( - "v1", new Instant(1), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), + "v1", Instant.ofEpochMilli(1), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), WindowedValues.of( - "v2", new Instant(2), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), + "v2", Instant.ofEpochMilli(2), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), WindowedValues.of( - "v3", new Instant(13), Arrays.asList(window(10, 20)), PaneInfo.NO_FIRING)); + "v3", Instant.ofEpochMilli(13), Arrays.asList(window(10, 20)), PaneInfo.NO_FIRING)); assertThat(result, hasSize(2)); @@ -156,12 +156,12 @@ public static void groupsElementsIntoSlidingWindowsWithMinTimestamp( "key", WindowedValues.of( "v1", - new Instant(5), + Instant.ofEpochMilli(5), Arrays.asList(window(-10, 10), window(0, 20)), PaneInfo.NO_FIRING), WindowedValues.of( "v2", - new Instant(15), + Instant.ofEpochMilli(15), Arrays.asList(window(0, 20), window(10, 30)), PaneInfo.NO_FIRING)); @@ -170,17 +170,17 @@ public static void groupsElementsIntoSlidingWindowsWithMinTimestamp( TimestampedValue>> item0 = getOnlyElementInWindow(result, window(-10, 10)); assertThat(item0.getValue().getValue(), contains("v1")); - assertThat(item0.getTimestamp(), equalTo(new Instant(5))); + assertThat(item0.getTimestamp(), equalTo(Instant.ofEpochMilli(5))); TimestampedValue>> item1 = getOnlyElementInWindow(result, window(0, 20)); assertThat(item1.getValue().getValue(), containsInAnyOrder("v1", "v2")); - assertThat(item1.getTimestamp(), equalTo(new Instant(5))); + assertThat(item1.getTimestamp(), equalTo(Instant.ofEpochMilli(5))); TimestampedValue>> item2 = getOnlyElementInWindow(result, window(10, 30)); assertThat(item2.getValue().getValue(), contains("v2")); - assertThat(item2.getTimestamp(), equalTo(new Instant(15))); + assertThat(item2.getTimestamp(), equalTo(Instant.ofEpochMilli(15))); } /** @@ -205,17 +205,17 @@ public static void combinesElementsInSlidingWindows( "k", WindowedValues.of( 1L, - new Instant(5), + Instant.ofEpochMilli(5), Arrays.asList(window(-10, 10), window(0, 20)), PaneInfo.NO_FIRING), WindowedValues.of( 2L, - new Instant(15), + Instant.ofEpochMilli(15), Arrays.asList(window(0, 20), window(10, 30)), PaneInfo.NO_FIRING), WindowedValues.of( 4L, - new Instant(18), + Instant.ofEpochMilli(18), Arrays.asList(window(0, 20), window(10, 30)), PaneInfo.NO_FIRING)); @@ -224,17 +224,17 @@ public static void combinesElementsInSlidingWindows( TimestampedValue> item0 = getOnlyElementInWindow(result, window(-10, 10)); assertThat(item0.getValue().getKey(), equalTo("k")); assertThat(item0.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(1L)))); - assertThat(item0.getTimestamp(), equalTo(new Instant(5L))); + assertThat(item0.getTimestamp(), equalTo(Instant.ofEpochMilli(5L))); TimestampedValue> item1 = getOnlyElementInWindow(result, window(0, 20)); assertThat(item1.getValue().getKey(), equalTo("k")); assertThat(item1.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(1L, 2L, 4L)))); - assertThat(item1.getTimestamp(), equalTo(new Instant(5L))); + assertThat(item1.getTimestamp(), equalTo(Instant.ofEpochMilli(5L))); TimestampedValue> item2 = getOnlyElementInWindow(result, window(10, 30)); assertThat(item2.getValue().getKey(), equalTo("k")); assertThat(item2.getValue().getValue(), equalTo(combineFn.apply(ImmutableList.of(2L, 4L)))); - assertThat(item2.getTimestamp(), equalTo(new Instant(15L))); + assertThat(item2.getTimestamp(), equalTo(Instant.ofEpochMilli(15L))); } /** @@ -253,11 +253,11 @@ public static void groupsIntoOverlappingNonmergingWindows( windowingStrategy, "key", WindowedValues.of( - "v1", new Instant(1), Arrays.asList(window(0, 5)), PaneInfo.NO_FIRING), + "v1", Instant.ofEpochMilli(1), Arrays.asList(window(0, 5)), PaneInfo.NO_FIRING), WindowedValues.of( - "v2", new Instant(4), Arrays.asList(window(1, 5)), PaneInfo.NO_FIRING), + "v2", Instant.ofEpochMilli(4), Arrays.asList(window(1, 5)), PaneInfo.NO_FIRING), WindowedValues.of( - "v3", new Instant(4), Arrays.asList(window(0, 5)), PaneInfo.NO_FIRING)); + "v3", Instant.ofEpochMilli(4), Arrays.asList(window(0, 5)), PaneInfo.NO_FIRING)); assertThat(result, hasSize(2)); @@ -285,11 +285,11 @@ public static void groupsElementsInMergedSessions( windowingStrategy, "key", WindowedValues.of( - "v1", new Instant(0), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), + "v1", Instant.ofEpochMilli(0), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), WindowedValues.of( - "v2", new Instant(5), Arrays.asList(window(5, 15)), PaneInfo.NO_FIRING), + "v2", Instant.ofEpochMilli(5), Arrays.asList(window(5, 15)), PaneInfo.NO_FIRING), WindowedValues.of( - "v3", new Instant(15), Arrays.asList(window(15, 25)), PaneInfo.NO_FIRING)); + "v3", Instant.ofEpochMilli(15), Arrays.asList(window(15, 25)), PaneInfo.NO_FIRING)); assertThat(result, hasSize(2)); @@ -321,10 +321,12 @@ public static void combinesElementsPerSession( gabwFactory, windowingStrategy, "k", - WindowedValues.of(1L, new Instant(0), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), - WindowedValues.of(2L, new Instant(5), Arrays.asList(window(5, 15)), PaneInfo.NO_FIRING), WindowedValues.of( - 4L, new Instant(15), Arrays.asList(window(15, 25)), PaneInfo.NO_FIRING)); + 1L, Instant.ofEpochMilli(0), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), + WindowedValues.of( + 2L, Instant.ofEpochMilli(5), Arrays.asList(window(5, 15)), PaneInfo.NO_FIRING), + WindowedValues.of( + 4L, Instant.ofEpochMilli(15), Arrays.asList(window(15, 25)), PaneInfo.NO_FIRING)); assertThat(result, hasSize(2)); @@ -357,11 +359,11 @@ public static void groupsElementsIntoFixedWindowsWithEndOfWindowTimestamp( windowingStrategy, "key", WindowedValues.of( - "v1", new Instant(1), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), + "v1", Instant.ofEpochMilli(1), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), WindowedValues.of( - "v2", new Instant(2), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), + "v2", Instant.ofEpochMilli(2), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), WindowedValues.of( - "v3", new Instant(13), Arrays.asList(window(10, 20)), PaneInfo.NO_FIRING)); + "v3", Instant.ofEpochMilli(13), Arrays.asList(window(10, 20)), PaneInfo.NO_FIRING)); assertThat(result, hasSize(2)); @@ -394,23 +396,23 @@ public static void groupsElementsIntoFixedWindowsWithLatestTimestamp( windowingStrategy, "k", WindowedValues.of( - "v1", new Instant(1), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), + "v1", Instant.ofEpochMilli(1), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), WindowedValues.of( - "v2", new Instant(2), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), + "v2", Instant.ofEpochMilli(2), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), WindowedValues.of( - "v3", new Instant(13), Arrays.asList(window(10, 20)), PaneInfo.NO_FIRING)); + "v3", Instant.ofEpochMilli(13), Arrays.asList(window(10, 20)), PaneInfo.NO_FIRING)); assertThat(result, hasSize(2)); TimestampedValue>> item0 = getOnlyElementInWindow(result, window(0, 10)); assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v2")); - assertThat(item0.getTimestamp(), equalTo(new Instant(2))); + assertThat(item0.getTimestamp(), equalTo(Instant.ofEpochMilli(2))); TimestampedValue>> item1 = getOnlyElementInWindow(result, window(10, 20)); assertThat(item1.getValue().getValue(), contains("v3")); - assertThat(item1.getTimestamp(), equalTo(new Instant(13))); + assertThat(item1.getTimestamp(), equalTo(Instant.ofEpochMilli(13))); } /** @@ -430,11 +432,11 @@ public static void groupsElementsInMergedSessionsWithEndOfWindowTimestamp( windowingStrategy, "k", WindowedValues.of( - "v1", new Instant(0), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), + "v1", Instant.ofEpochMilli(0), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), WindowedValues.of( - "v2", new Instant(5), Arrays.asList(window(5, 15)), PaneInfo.NO_FIRING), + "v2", Instant.ofEpochMilli(5), Arrays.asList(window(5, 15)), PaneInfo.NO_FIRING), WindowedValues.of( - "v3", new Instant(15), Arrays.asList(window(15, 25)), PaneInfo.NO_FIRING)); + "v3", Instant.ofEpochMilli(15), Arrays.asList(window(15, 25)), PaneInfo.NO_FIRING)); assertThat(result, hasSize(2)); @@ -467,11 +469,11 @@ public static void groupsElementsInMergedSessionsWithLatestTimestamp( windowingStrategy, "k", WindowedValues.of( - "v1", new Instant(0), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), + "v1", Instant.ofEpochMilli(0), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), WindowedValues.of( - "v2", new Instant(5), Arrays.asList(window(5, 15)), PaneInfo.NO_FIRING), + "v2", Instant.ofEpochMilli(5), Arrays.asList(window(5, 15)), PaneInfo.NO_FIRING), WindowedValues.of( - "v3", new Instant(15), Arrays.asList(unmergedWindow), PaneInfo.NO_FIRING)); + "v3", Instant.ofEpochMilli(15), Arrays.asList(unmergedWindow), PaneInfo.NO_FIRING)); assertThat(result, hasSize(2)); @@ -479,12 +481,12 @@ public static void groupsElementsInMergedSessionsWithLatestTimestamp( TimestampedValue>> item0 = getOnlyElementInWindow(result, mergedWindow); assertThat(item0.getValue().getValue(), containsInAnyOrder("v1", "v2")); - assertThat(item0.getTimestamp(), equalTo(new Instant(5))); + assertThat(item0.getTimestamp(), equalTo(Instant.ofEpochMilli(5))); TimestampedValue>> item1 = getOnlyElementInWindow(result, unmergedWindow); assertThat(item1.getValue().getValue(), contains("v3")); - assertThat(item1.getTimestamp(), equalTo(new Instant(15))); + assertThat(item1.getTimestamp(), equalTo(Instant.ofEpochMilli(15))); } /** @@ -506,10 +508,12 @@ public static void combinesElementsPerSessionWithEndOfWindowTimestamp( gabwFactory, windowingStrategy, "k", - WindowedValues.of(1L, new Instant(0), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), - WindowedValues.of(2L, new Instant(5), Arrays.asList(window(5, 15)), PaneInfo.NO_FIRING), WindowedValues.of( - 4L, new Instant(15), Arrays.asList(secondWindow), PaneInfo.NO_FIRING)); + 1L, Instant.ofEpochMilli(0), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), + WindowedValues.of( + 2L, Instant.ofEpochMilli(5), Arrays.asList(window(5, 15)), PaneInfo.NO_FIRING), + WindowedValues.of( + 4L, Instant.ofEpochMilli(15), Arrays.asList(secondWindow), PaneInfo.NO_FIRING)); assertThat(result, hasSize(2)); @@ -558,7 +562,7 @@ List>> runGABW( } private static BoundedWindow window(long start, long end) { - return new IntervalWindow(new Instant(start), new Instant(end)); + return new IntervalWindow(Instant.ofEpochMilli(start), Instant.ofEpochMilli(end)); } private static final class CachingStateInternalsFactory implements StateInternalsFactory { diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServerTest.java index d417d7d3417c..a2ced5f83b2b 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillServerTest.java @@ -341,8 +341,8 @@ public void onCompleted() { long serializedWorkItemSize, ImmutableList getWorkStreamLatencies) -> { latch.countDown(); - assertEquals(inputDataWatermark, new Instant(18)); - assertEquals(synchronizedProcessingTime, new Instant(17)); + assertEquals(inputDataWatermark, Instant.ofEpochMilli(18)); + assertEquals(synchronizedProcessingTime, Instant.ofEpochMilli(17)); assertEquals(workItem.getKey(), ByteString.copyFromUtf8("somewhat_long_key")); }); assertTrue(latch.await(30, TimeUnit.SECONDS)); @@ -475,8 +475,8 @@ public void onCompleted() { WorkItem workItem, long serializedWorkItemSize, ImmutableList getWorkStreamLatencies) -> { - assertEquals(inputDataWatermark, new Instant(18)); - assertEquals(synchronizedProcessingTime, new Instant(17)); + assertEquals(inputDataWatermark, Instant.ofEpochMilli(18)); + assertEquals(synchronizedProcessingTime, Instant.ofEpochMilli(17)); assertEquals(workItem.getKey(), ByteString.copyFromUtf8("somewhat_long_key")); assertTrue(drainMode); assertTrue(sentResponseIds.containsKey(workItem.getWorkToken())); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateCacheTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateCacheTest.java index bbb8e4c93c07..1a3b8f0dcde3 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateCacheTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateCacheTest.java @@ -137,13 +137,14 @@ public String toString() { private static StateNamespace windowNamespace(long start) { return StateNamespaces.window( - IntervalWindow.getCoder(), new IntervalWindow(new Instant(start), new Instant(start + 1))); + IntervalWindow.getCoder(), + new IntervalWindow(Instant.ofEpochMilli(start), Instant.ofEpochMilli(start + 1))); } private static StateNamespace triggerNamespace(long start, int triggerIdx) { return StateNamespaces.windowAndTrigger( IntervalWindow.getCoder(), - new IntervalWindow(new Instant(start), new Instant(start + 1)), + new IntervalWindow(Instant.ofEpochMilli(start), Instant.ofEpochMilli(start + 1)), triggerIdx); } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateInternalsTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateInternalsTest.java index 7a06d3a29493..024a5650ef21 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateInternalsTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateInternalsTest.java @@ -2954,16 +2954,16 @@ public void testWatermarkAddBeforeReadEarliest() throws Exception { bag.readLater(); - bag.add(new Instant(3000)); - waitAndSet(future, new Instant(2000), 200); - assertThat(bag.read(), Matchers.equalTo(new Instant(2000))); + bag.add(Instant.ofEpochMilli(3000)); + waitAndSet(future, Instant.ofEpochMilli(2000), 200); + assertThat(bag.read(), Matchers.equalTo(Instant.ofEpochMilli(2000))); Mockito.verify(mockReader, times(2)).watermarkFuture(key(NAMESPACE, "watermark"), STATE_FAMILY); Mockito.verifyNoMoreInteractions(mockReader); // Adding another value doesn't create another future, but does update the result. - bag.add(new Instant(1000)); - assertThat(bag.read(), Matchers.equalTo(new Instant(1000))); + bag.add(Instant.ofEpochMilli(1000)); + assertThat(bag.read(), Matchers.equalTo(Instant.ofEpochMilli(1000))); Mockito.verifyNoMoreInteractions(mockReader); } @@ -2981,16 +2981,16 @@ public void testWatermarkAddBeforeReadLatest() throws Exception { // Actually reading it will request another future, and get the same one, from // WindmillStateReader - bag.add(new Instant(3000)); - waitAndSet(future, new Instant(2000), 200); - assertThat(bag.read(), Matchers.equalTo(new Instant(3000))); + bag.add(Instant.ofEpochMilli(3000)); + waitAndSet(future, Instant.ofEpochMilli(2000), 200); + assertThat(bag.read(), Matchers.equalTo(Instant.ofEpochMilli(3000))); Mockito.verify(mockReader, times(2)).watermarkFuture(key(NAMESPACE, "watermark"), STATE_FAMILY); Mockito.verifyNoMoreInteractions(mockReader); // Adding another value doesn't create another future, but does update the result. - bag.add(new Instant(3000)); - assertThat(bag.read(), Matchers.equalTo(new Instant(3000))); + bag.add(Instant.ofEpochMilli(3000)); + assertThat(bag.read(), Matchers.equalTo(Instant.ofEpochMilli(3000))); Mockito.verifyNoMoreInteractions(mockReader); } @@ -3006,17 +3006,17 @@ public void testWatermarkAddBeforeReadEndOfWindow() throws Exception { // Requests a future once bag.readLater(); - bag.add(new Instant(3000)); - waitAndSet(future, new Instant(3000), 200); + bag.add(Instant.ofEpochMilli(3000)); + waitAndSet(future, Instant.ofEpochMilli(3000), 200); // read() requests a future again, receiving the same one - assertThat(bag.read(), Matchers.equalTo(new Instant(3000))); + assertThat(bag.read(), Matchers.equalTo(Instant.ofEpochMilli(3000))); Mockito.verify(mockReader, times(2)).watermarkFuture(key(NAMESPACE, "watermark"), STATE_FAMILY); Mockito.verifyNoMoreInteractions(mockReader); // Adding another value doesn't create another future, but does update the result. - bag.add(new Instant(3000)); - assertThat(bag.read(), Matchers.equalTo(new Instant(3000))); + bag.add(Instant.ofEpochMilli(3000)); + assertThat(bag.read(), Matchers.equalTo(Instant.ofEpochMilli(3000))); Mockito.verifyNoMoreInteractions(mockReader); } @@ -3030,8 +3030,8 @@ public void testWatermarkClearBeforeRead() throws Exception { bag.clear(); assertThat(bag.read(), Matchers.nullValue()); - bag.add(new Instant(300)); - assertThat(bag.read(), Matchers.equalTo(new Instant(300))); + bag.add(Instant.ofEpochMilli(300)); + assertThat(bag.read(), Matchers.equalTo(Instant.ofEpochMilli(300))); // Shouldn't need to read from windmill because the value is already available. Mockito.verifyNoMoreInteractions(mockReader); @@ -3043,8 +3043,8 @@ public void testWatermarkPersistEarliest() throws Exception { StateTags.watermarkStateInternal("watermark", TimestampCombiner.EARLIEST); WatermarkHoldState bag = underTest.state(NAMESPACE, addr); - bag.add(new Instant(1000)); - bag.add(new Instant(2000)); + bag.add(Instant.ofEpochMilli(1000)); + bag.add(Instant.ofEpochMilli(2000)); Windmill.WorkItemCommitRequest.Builder commitBuilder = Windmill.WorkItemCommitRequest.newBuilder(); @@ -3067,8 +3067,8 @@ public void testWatermarkPersistLatestEmpty() throws Exception { StateTags.watermarkStateInternal("watermark", TimestampCombiner.LATEST); WatermarkHoldState hold = underTest.state(NAMESPACE, addr); - hold.add(new Instant(1000)); - hold.add(new Instant(2000)); + hold.add(Instant.ofEpochMilli(1000)); + hold.add(Instant.ofEpochMilli(2000)); when(mockReader.watermarkFuture(key(NAMESPACE, "watermark"), STATE_FAMILY)) .thenReturn(Futures.immediateFuture(null)); @@ -3095,11 +3095,11 @@ public void testWatermarkPersistLatestWindmillWins() throws Exception { StateTags.watermarkStateInternal("watermark", TimestampCombiner.LATEST); WatermarkHoldState hold = underTest.state(NAMESPACE, addr); - hold.add(new Instant(1000)); - hold.add(new Instant(2000)); + hold.add(Instant.ofEpochMilli(1000)); + hold.add(Instant.ofEpochMilli(2000)); when(mockReader.watermarkFuture(key(NAMESPACE, "watermark"), STATE_FAMILY)) - .thenReturn(Futures.immediateFuture(new Instant(4000))); + .thenReturn(Futures.immediateFuture(Instant.ofEpochMilli(4000))); Windmill.WorkItemCommitRequest.Builder commitBuilder = Windmill.WorkItemCommitRequest.newBuilder(); @@ -3123,11 +3123,11 @@ public void testWatermarkPersistLatestLocalAdditionsWin() throws Exception { StateTags.watermarkStateInternal("watermark", TimestampCombiner.LATEST); WatermarkHoldState hold = underTest.state(NAMESPACE, addr); - hold.add(new Instant(1000)); - hold.add(new Instant(2000)); + hold.add(Instant.ofEpochMilli(1000)); + hold.add(Instant.ofEpochMilli(2000)); when(mockReader.watermarkFuture(key(NAMESPACE, "watermark"), STATE_FAMILY)) - .thenReturn(Futures.immediateFuture(new Instant(500))); + .thenReturn(Futures.immediateFuture(Instant.ofEpochMilli(500))); Windmill.WorkItemCommitRequest.Builder commitBuilder = Windmill.WorkItemCommitRequest.newBuilder(); @@ -3151,8 +3151,8 @@ public void testWatermarkPersistEndOfWindow() throws Exception { StateTags.watermarkStateInternal("watermark", TimestampCombiner.END_OF_WINDOW); WatermarkHoldState hold = underTest.state(NAMESPACE, addr); - hold.add(new Instant(2000)); - hold.add(new Instant(2000)); + hold.add(Instant.ofEpochMilli(2000)); + hold.add(Instant.ofEpochMilli(2000)); Windmill.WorkItemCommitRequest.Builder commitBuilder = Windmill.WorkItemCommitRequest.newBuilder(); @@ -3176,10 +3176,10 @@ public void testWatermarkClearPersist() throws Exception { StateTags.watermarkStateInternal("watermark", TimestampCombiner.EARLIEST); WatermarkHoldState hold = underTest.state(NAMESPACE, addr); - hold.add(new Instant(500)); + hold.add(Instant.ofEpochMilli(500)); hold.clear(); - hold.add(new Instant(1000)); - hold.add(new Instant(2000)); + hold.add(Instant.ofEpochMilli(1000)); + hold.add(Instant.ofEpochMilli(2000)); Windmill.WorkItemCommitRequest.Builder commitBuilder = Windmill.WorkItemCommitRequest.newBuilder(); @@ -3203,7 +3203,7 @@ public void testWatermarkPersistEmpty() throws Exception { StateTags.watermarkStateInternal("watermark", TimestampCombiner.EARLIEST); WatermarkHoldState bag = underTest.state(NAMESPACE, addr); - bag.add(new Instant(500)); + bag.add(Instant.ofEpochMilli(500)); bag.clear(); Windmill.WorkItemCommitRequest.Builder commitBuilder = @@ -3440,9 +3440,9 @@ public void testCachedWatermarkHold() throws Exception { hold.readLater(); - hold.add(new Instant(3000)); - waitAndSet(future, new Instant(2000), 200); - assertThat(hold.read(), Matchers.equalTo(new Instant(2000))); + hold.add(Instant.ofEpochMilli(3000)); + waitAndSet(future, Instant.ofEpochMilli(2000), 200); + assertThat(hold.read(), Matchers.equalTo(Instant.ofEpochMilli(2000))); underTest.persist(Windmill.WorkItemCommitRequest.newBuilder()); @@ -3450,7 +3450,7 @@ public void testCachedWatermarkHold() throws Exception { resetUnderTest(); hold = underTest.state(NAMESPACE, addr); - assertThat(hold.read(), Matchers.equalTo(new Instant(2000))); + assertThat(hold.read(), Matchers.equalTo(Instant.ofEpochMilli(2000))); hold.clear(); underTest.persist(Windmill.WorkItemCommitRequest.newBuilder()); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReaderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReaderTest.java index 1611fdac25dc..43652030eda1 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReaderTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReaderTest.java @@ -1459,7 +1459,7 @@ public void testReadWatermark() throws Exception { Instant result = future.get(); Mockito.verify(mockWindmill).getStateData(COMPUTATION, expectedRequest.build()); - assertThat(result, Matchers.equalTo(new Instant(5000))); + assertThat(result, Matchers.equalTo(Instant.ofEpochMilli(5000))); assertNoReader(future); } @@ -1508,7 +1508,7 @@ public void testBatching() throws Exception { assertThat(keyedRequest.getWatermarkHoldsToFetch(0).getTag(), Matchers.equalTo(STATE_KEY_2)); // Verify the values returned to the user. - assertThat(result, Matchers.equalTo(new Instant(5000))); + assertThat(result, Matchers.equalTo(Instant.ofEpochMilli(5000))); Mockito.verifyNoMoreInteractions(mockWindmill); assertThat(bagFuture.get(), Matchers.contains(5, 100)); @@ -1740,7 +1740,7 @@ public void testBatchingCoderExceptions() throws Exception { assertThat(e.toString(), Matchers.containsString("Error parsing tag value prefix")); } - assertThat(watermarkFuture.get(), Matchers.equalTo(new Instant(5000))); + assertThat(watermarkFuture.get(), Matchers.equalTo(Instant.ofEpochMilli(5000))); } /** diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillTagEncodingV1Test.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillTagEncodingV1Test.java index fcac966ff963..a6d404183a76 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillTagEncodingV1Test.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillTagEncodingV1Test.java @@ -63,17 +63,17 @@ public class WindmillTagEncodingV1Test { IntervalWindow.getCoder(), StateNamespaces.window( IntervalWindow.getCoder(), - new IntervalWindow(new Instant(13), new Instant(47))))); + new IntervalWindow(Instant.ofEpochMilli(13), Instant.ofEpochMilli(47))))); private static final List TEST_TIMESTAMPS = ImmutableList.of( BoundedWindow.TIMESTAMP_MIN_VALUE, BoundedWindow.TIMESTAMP_MAX_VALUE, GlobalWindow.INSTANCE.maxTimestamp(), - new Instant(0), - new Instant(127), + Instant.ofEpochMilli(0), + Instant.ofEpochMilli(127), // The encoding of Instant(716000) ends with '+'. - new Instant(716001)); + Instant.ofEpochMilli(716001)); private static final List TEST_STATE_FAMILIES = ImmutableList.of("", "F24"); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillTagEncodingV2Test.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillTagEncodingV2Test.java index 4bc2bba79b12..3bbbab84c8c9 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillTagEncodingV2Test.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillTagEncodingV2Test.java @@ -64,7 +64,7 @@ public class WindmillTagEncodingV2Test { private static final IntervalWindow INTERVAL_WINDOW = - new IntervalWindow(new Instant(10), new Instant(20)); + new IntervalWindow(Instant.ofEpochMilli(10), Instant.ofEpochMilli(20)); private static final CustomWindow CUSTOM_WINDOW = new CustomWindow(INTERVAL_WINDOW); @@ -317,14 +317,14 @@ public void testTimerTag() { TIMER_ID, TIMER_FAMILY_ID, namespace, - new Instant(123), - new Instant(456), + Instant.ofEpochMilli(123), + Instant.ofEpochMilli(456), timeDomain) : TimerData.of( TIMER_ID, namespace, - new Instant(123), - new Instant(456), + Instant.ofEpochMilli(123), + Instant.ofEpochMilli(456), timeDomain, CausedByDrain.NORMAL); assertEquals( @@ -468,8 +468,8 @@ public void testTimerHoldTagUsesTimerTag() { TIMER_ID, TIMER_FAMILY_ID, GLOBAL_NAMESPACE, - new Instant(123), - new Instant(456), + Instant.ofEpochMilli(123), + Instant.ofEpochMilli(456), TimeDomain.EVENT_TIME); byte[] bytes = new byte[16]; ThreadLocalRandom.current().nextBytes(bytes); diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/CommonCoderTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/CommonCoderTest.java index a79b0996cbb2..86a74a2f350c 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/CommonCoderTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/CommonCoderTest.java @@ -325,13 +325,13 @@ private static Object convertValue(Object value, CommonCoder coderSpec, Coder co convertValue(kvMap.get("userKey"), coderSpec.getComponents().get(0), keyCoder), (String) kvMap.get("dynamicTimerTag"), windows, - new Instant(((Number) kvMap.get("fireTimestamp")).longValue()), - new Instant(((Number) kvMap.get("holdTimestamp")).longValue()), + Instant.ofEpochMilli(((Number) kvMap.get("fireTimestamp")).longValue()), + Instant.ofEpochMilli(((Number) kvMap.get("holdTimestamp")).longValue()), paneInfo, CausedByDrain.NORMAL); // todo - add tests once causedByDrain is added to proto } else if (s.equals(getUrn(StandardCoders.Enum.INTERVAL_WINDOW))) { Map kvMap = (Map) value; - Instant end = new Instant(((Number) kvMap.get("end")).longValue()); + Instant end = Instant.ofEpochMilli(((Number) kvMap.get("end")).longValue()); Duration span = Duration.millis(((Number) kvMap.get("span")).longValue()); return new IntervalWindow(end.minus(span), span); } else if (s.equals(getUrn(StandardCoders.Enum.ITERABLE)) @@ -353,7 +353,7 @@ private static Object convertValue(Object value, CommonCoder coderSpec, Coder co Coder windowCoder = ((WindowedValues.FullWindowedValueCoder) coder).getWindowCoder(); Object windowValue = convertValue(kvMap.get("value"), coderSpec.getComponents().get(0), valueCoder); - Instant timestamp = new Instant(((Number) kvMap.get("timestamp")).longValue()); + Instant timestamp = Instant.ofEpochMilli(((Number) kvMap.get("timestamp")).longValue()); List windows = new ArrayList<>(); for (Object window : (List) kvMap.get("windows")) { windows.add( @@ -430,7 +430,7 @@ private static Object parseField(Object value, Schema.FieldType fieldType) { return (Boolean) value; case DATETIME: // convert shifted millis to epoch millis as in InstantCoder - return new Instant((Long) value + -9223372036854775808L); + return Instant.ofEpochMilli((Long) value + -9223372036854775808L); case DECIMAL: return new BigDecimal((String) value); case BYTES: diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/StatefulParDoP.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/StatefulParDoP.java index ed085e5e56b1..4adbf6375935 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/StatefulParDoP.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/StatefulParDoP.java @@ -333,7 +333,7 @@ public void advanceProcessingTimes() { } public void flushTimers(long watermark) { - Instant watermarkInstant = new Instant(watermark); + Instant watermarkInstant = Instant.ofEpochMilli(watermark); keyedTimerInternals .entrySet() .forEach( diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/WindowGroupP.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/WindowGroupP.java index 747df5a67bd3..b4d0a51515b9 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/WindowGroupP.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/WindowGroupP.java @@ -174,7 +174,7 @@ public boolean complete() { } private void advanceWatermark(long millis) { - this.latestWatermark = new Instant(millis); + this.latestWatermark = Instant.ofEpochMilli(millis); Instant now = Instant.now(); for (KeyManager m : keyManagers.values()) { m.advanceWatermark(latestWatermark, now); diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/OpAdapter.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/OpAdapter.java index f2eecbbbc9c7..070bb2a5af49 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/OpAdapter.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/OpAdapter.java @@ -130,7 +130,7 @@ public synchronized CompletionStage>> apply(OpMessage @Override public synchronized Collection> processWatermark(long time) { try { - op.processWatermark(new Instant(time), emitter); + op.processWatermark(Instant.ofEpochMilli(time), emitter); } catch (Exception e) { LOG.error( "Op {} threw an exception during processing watermark", this.getClass().getName(), e); diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaTimerInternalsFactory.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaTimerInternalsFactory.java index bd6547b805a4..a8326dd9a61c 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaTimerInternalsFactory.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaTimerInternalsFactory.java @@ -271,8 +271,8 @@ public void setTimer(TimerData timerData) { timerData.getNamespace(), timerData.getTimerId(), timerData.getTimerFamilyId(), - new Instant(lastTimestamp), - new Instant(lastTimestamp), + Instant.ofEpochMilli(lastTimestamp), + Instant.ofEpochMilli(lastTimestamp), timerData.getDomain()); } @@ -381,7 +381,7 @@ private void deleteTimer( @Override public Instant currentProcessingTime() { - return new Instant(); + return Instant.now(); } @Override @@ -651,8 +651,8 @@ static KeyedTimerData toKeyedTimerData( timerKey.getTimerId(), timerKey.getTimerFamilyId(), timerKey.getStateNamespace(), - new Instant(timestamp), - new Instant(timestamp), + Instant.ofEpochMilli(timestamp), + Instant.ofEpochMilli(timestamp), domain)); } diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ParDoBoundMultiTranslator.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ParDoBoundMultiTranslator.java index 6b4d464b8e7c..69a38ea337d3 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ParDoBoundMultiTranslator.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ParDoBoundMultiTranslator.java @@ -537,7 +537,8 @@ public Collection> apply(OpMessage message) { @Override public Collection> processWatermark(long watermark) { - return Collections.singletonList(OpMessage.ofSideInputWatermark(new Instant(watermark))); + return Collections.singletonList( + OpMessage.ofSideInputWatermark(Instant.ofEpochMilli(watermark))); } @Override diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/TranslationContext.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/TranslationContext.java index c5e984fbde07..8e83eeddabb8 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/TranslationContext.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/TranslationContext.java @@ -369,7 +369,7 @@ public StoreIdGenerator getStoreIdGenerator() { producer.send(id, new OutgoingMessageEnvelope(sysStream, 0, null, msg)); }; final WindowedValue windowedValue = - WindowedValues.timestampedValueInGlobalWindow("dummy", new Instant()); + WindowedValues.timestampedValueInGlobalWindow("dummy", Instant.now()); sendFn.accept(OpMessage.ofElement(windowedValue)); sendFn.accept(new WatermarkMessage(BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis())); diff --git a/runners/samza/src/test/java/org/apache/beam/runners/samza/metrics/TestSamzaRunnerWithTransformMetrics.java b/runners/samza/src/test/java/org/apache/beam/runners/samza/metrics/TestSamzaRunnerWithTransformMetrics.java index dc660ce85af5..984469787b56 100644 --- a/runners/samza/src/test/java/org/apache/beam/runners/samza/metrics/TestSamzaRunnerWithTransformMetrics.java +++ b/runners/samza/src/test/java/org/apache/beam/runners/samza/metrics/TestSamzaRunnerWithTransformMetrics.java @@ -176,9 +176,9 @@ public void testSamzaRunnerWithDefaultMetrics() { @Test public void testSamzaInputAndOutputMetricOp() { final WindowedValue windowedValue = - WindowedValues.timestampedValueInGlobalWindow("value-1", new Instant()); + WindowedValues.timestampedValueInGlobalWindow("value-1", Instant.now()); final WindowedValue windowedValue2 = - WindowedValues.timestampedValueInGlobalWindow("value-2", new Instant()); + WindowedValues.timestampedValueInGlobalWindow("value-2", Instant.now()); final WatermarkMessage watermarkMessage = new WatermarkMessage(BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis()); @@ -215,7 +215,8 @@ public void testSamzaInputAndOutputMetricOp() { inputMetricOp.processElement(windowedValue, opEmitter); inputMetricOp.processElement(windowedValue2, opEmitter); - inputMetricOp.processWatermark(new Instant(watermarkMessage.getTimestamp()), opEmitter); + inputMetricOp.processWatermark( + Instant.ofEpochMilli(watermarkMessage.getTimestamp()), opEmitter); // Input throughput must be updated assertEquals(2, inputCounter.getCount()); @@ -236,7 +237,8 @@ public void testSamzaInputAndOutputMetricOp() { outputMetricOp.processElement(windowedValue, opEmitter); outputMetricOp.processElement(windowedValue2, opEmitter); - outputMetricOp.processWatermark(new Instant(watermarkMessage.getTimestamp()), opEmitter); + outputMetricOp.processWatermark( + Instant.ofEpochMilli(watermarkMessage.getTimestamp()), opEmitter); // Output throughput must be updated assertEquals(2, outputCounter.getCount()); @@ -251,9 +253,9 @@ public void testSamzaInputAndOutputMetricOp() { @Test public void testSamzaInputAndOutputGBKMetricOp() { final WindowedValue windowedValue = - WindowedValues.timestampedValueInGlobalWindow("value-1", new Instant()); + WindowedValues.timestampedValueInGlobalWindow("value-1", Instant.now()); final WindowedValue windowedValue2 = - WindowedValues.timestampedValueInGlobalWindow("value-2", new Instant()); + WindowedValues.timestampedValueInGlobalWindow("value-2", Instant.now()); final WatermarkMessage watermarkMessage = new WatermarkMessage(BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis()); @@ -292,7 +294,8 @@ public void testSamzaInputAndOutputGBKMetricOp() { inputMetricOp.processElement(windowedValue, opEmitter); inputMetricOp.processElement(windowedValue2, opEmitter); - inputMetricOp.processWatermark(new Instant(watermarkMessage.getTimestamp()), opEmitter); + inputMetricOp.processWatermark( + Instant.ofEpochMilli(watermarkMessage.getTimestamp()), opEmitter); // Input throughput must be updated assertEquals(2, inputCounter.getCount()); @@ -309,7 +312,8 @@ public void testSamzaInputAndOutputGBKMetricOp() { outputMetricOp.processElement(windowedValue, opEmitter); outputMetricOp.processElement(windowedValue2, opEmitter); - outputMetricOp.processWatermark(new Instant(watermarkMessage.getTimestamp()), opEmitter); + outputMetricOp.processWatermark( + Instant.ofEpochMilli(watermarkMessage.getTimestamp()), opEmitter); // Output throughput must be updated assertEquals(2, outputCounter.getCount()); diff --git a/runners/samza/src/test/java/org/apache/beam/runners/samza/metrics/TestSamzaTransformMetricsRegistry.java b/runners/samza/src/test/java/org/apache/beam/runners/samza/metrics/TestSamzaTransformMetricsRegistry.java index f94543ef809e..360604de59a2 100644 --- a/runners/samza/src/test/java/org/apache/beam/runners/samza/metrics/TestSamzaTransformMetricsRegistry.java +++ b/runners/samza/src/test/java/org/apache/beam/runners/samza/metrics/TestSamzaTransformMetricsRegistry.java @@ -125,14 +125,14 @@ public void testSamzaTransformMetricsRegistryForDataShuffleOperators() { new BoundedWindow() { @Override public Instant maxTimestamp() { - return new Instant(2048L); + return Instant.ofEpochMilli(2048L); } }; final BoundedWindow second = new BoundedWindow() { @Override public Instant maxTimestamp() { - return new Instant(689743L); + return Instant.ofEpochMilli(689743L); } }; diff --git a/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/GroupByKeyOpTest.java b/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/GroupByKeyOpTest.java index 73454cc95421..54cdd1105983 100644 --- a/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/GroupByKeyOpTest.java +++ b/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/GroupByKeyOpTest.java @@ -65,11 +65,11 @@ public static void beforeClass() { public void testDefaultGbk() { TestStream.Builder testStream = TestStream.create(VarIntCoder.of()) - .addElements(TimestampedValue.of(1, new Instant(1000))) - .addElements(TimestampedValue.of(2, new Instant(2000))) - .advanceWatermarkTo(new Instant(3000)) - .addElements(TimestampedValue.of(10, new Instant(1000))) - .advanceWatermarkTo(new Instant(10000)); + .addElements(TimestampedValue.of(1, Instant.ofEpochMilli(1000))) + .addElements(TimestampedValue.of(2, Instant.ofEpochMilli(2000))) + .advanceWatermarkTo(Instant.ofEpochMilli(3000)) + .addElements(TimestampedValue.of(10, Instant.ofEpochMilli(1000))) + .advanceWatermarkTo(Instant.ofEpochMilli(10000)); PCollection aggregated = pipeline @@ -88,11 +88,11 @@ public void testDefaultGbk() { public void testDropLateDataNonKeyed() { TestStream.Builder testStream = TestStream.create(VarIntCoder.of()) - .addElements(TimestampedValue.of(1, new Instant(1000))) - .addElements(TimestampedValue.of(2, new Instant(2000))) - .advanceWatermarkTo(new Instant(3000)) - .addElements(TimestampedValue.of(10, new Instant(1000))) - .advanceWatermarkTo(new Instant(10000)); + .addElements(TimestampedValue.of(1, Instant.ofEpochMilli(1000))) + .addElements(TimestampedValue.of(2, Instant.ofEpochMilli(2000))) + .advanceWatermarkTo(Instant.ofEpochMilli(3000)) + .addElements(TimestampedValue.of(10, Instant.ofEpochMilli(1000))) + .advanceWatermarkTo(Instant.ofEpochMilli(10000)); PCollection aggregated = dropLateDataPipeline @@ -111,12 +111,12 @@ public void testDropLateDataNonKeyed() { public void testDropLateDataKeyed() { TestStream.Builder> testStream = TestStream.create(KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of())) - .addElements(TimestampedValue.of(KV.of("a", 1), new Instant(1000))) - .addElements(TimestampedValue.of(KV.of("b", 2), new Instant(2000))) - .addElements(TimestampedValue.of(KV.of("a", 3), new Instant(2500))) - .advanceWatermarkTo(new Instant(3000)) - .addElements(TimestampedValue.of(KV.of("a", 10), new Instant(1000))) - .advanceWatermarkTo(new Instant(10000)); + .addElements(TimestampedValue.of(KV.of("a", 1), Instant.ofEpochMilli(1000))) + .addElements(TimestampedValue.of(KV.of("b", 2), Instant.ofEpochMilli(2000))) + .addElements(TimestampedValue.of(KV.of("a", 3), Instant.ofEpochMilli(2500))) + .advanceWatermarkTo(Instant.ofEpochMilli(3000)) + .addElements(TimestampedValue.of(KV.of("a", 10), Instant.ofEpochMilli(1000))) + .advanceWatermarkTo(Instant.ofEpochMilli(10000)); PCollection> aggregated = dropLateDataPipeline diff --git a/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/PortableBundleManagerTest.java b/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/PortableBundleManagerTest.java index 522e146d21e6..c7d8a07de965 100644 --- a/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/PortableBundleManagerTest.java +++ b/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/PortableBundleManagerTest.java @@ -124,7 +124,7 @@ public void testWhenSignalFailureThenResetBundle() throws Exception { @Test public void testProcessWatermarkWhenBundleNotStarted() { - Instant watermark = new Instant(); + Instant watermark = Instant.now(); portableBundleManager = new PortableBundleManager<>(bundleProgressListener, 4, 1, bundleTimerScheduler, TIMER_ID); portableBundleManager.processWatermark(watermark, emitter); @@ -133,7 +133,7 @@ public void testProcessWatermarkWhenBundleNotStarted() { @Test public void testQueueWatermarkWhenBundleStarted() { - Instant watermark = new Instant(); + Instant watermark = Instant.now(); portableBundleManager = new PortableBundleManager<>(bundleProgressListener, 1, 1, bundleTimerScheduler, TIMER_ID); diff --git a/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/SamzaTimerInternalsFactoryTest.java b/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/SamzaTimerInternalsFactoryTest.java index c7998a281f7c..49479f7685eb 100644 --- a/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/SamzaTimerInternalsFactoryTest.java +++ b/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/SamzaTimerInternalsFactoryTest.java @@ -154,8 +154,8 @@ public void testEventTimeTimers() { TimerInternals.TimerData.of( "timer1", nameSpace, - new Instant(10), - new Instant(10), + Instant.ofEpochMilli(10), + Instant.ofEpochMilli(10), TimeDomain.EVENT_TIME, CausedByDrain.NORMAL); timerInternals.setTimer(timer1); @@ -164,22 +164,22 @@ public void testEventTimeTimers() { TimerInternals.TimerData.of( "timer2", nameSpace, - new Instant(100), - new Instant(100), + Instant.ofEpochMilli(100), + Instant.ofEpochMilli(100), TimeDomain.EVENT_TIME, CausedByDrain.NORMAL); timerInternals.setTimer(timer2); - timerInternalsFactory.setInputWatermark(new Instant(5)); + timerInternalsFactory.setInputWatermark(Instant.ofEpochMilli(5)); Collection> readyTimers = timerInternalsFactory.removeReadyTimers(); assertTrue(readyTimers.isEmpty()); - timerInternalsFactory.setInputWatermark(new Instant(20)); + timerInternalsFactory.setInputWatermark(Instant.ofEpochMilli(20)); readyTimers = timerInternalsFactory.removeReadyTimers(); assertEquals(1, readyTimers.size()); assertEquals(timer1, readyTimers.iterator().next().getTimerData()); - timerInternalsFactory.setInputWatermark(new Instant(150)); + timerInternalsFactory.setInputWatermark(Instant.ofEpochMilli(150)); readyTimers = timerInternalsFactory.removeReadyTimers(); assertEquals(1, readyTimers.size()); assertEquals(timer2, readyTimers.iterator().next().getTimerData()); @@ -203,8 +203,8 @@ public void testRestoreEventBufferSize() throws Exception { TimerInternals.TimerData.of( "timer1", nameSpace, - new Instant(10), - new Instant(10), + Instant.ofEpochMilli(10), + Instant.ofEpochMilli(10), TimeDomain.EVENT_TIME, CausedByDrain.NORMAL); timerInternals.setTimer(timer1); @@ -218,7 +218,7 @@ public void testRestoreEventBufferSize() throws Exception { createTimerInternalsFactory(null, "timer", pipelineOptions, store); assertEquals(1, restoredFactory.getEventTimeBuffer().size()); - restoredFactory.setInputWatermark(new Instant(150)); + restoredFactory.setInputWatermark(Instant.ofEpochMilli(150)); Collection> readyTimers = restoredFactory.removeReadyTimers(); assertEquals(1, readyTimers.size()); @@ -229,8 +229,8 @@ public void testRestoreEventBufferSize() throws Exception { TimerInternals.TimerData.of( "timer2", nameSpace, - new Instant(200), - new Instant(200), + Instant.ofEpochMilli(200), + Instant.ofEpochMilli(200), TimeDomain.EVENT_TIME, CausedByDrain.NORMAL); restoredTimerInternals.setTimer(timer2); @@ -241,7 +241,7 @@ public void testRestoreEventBufferSize() throws Exception { readyTimers = restoredFactory.removeReadyTimers(); assertEquals(0, readyTimers.size()); - restoredFactory.setInputWatermark(new Instant(250)); + restoredFactory.setInputWatermark(Instant.ofEpochMilli(250)); // Timer 2 should be ready readyTimers = restoredFactory.removeReadyTimers(); @@ -270,8 +270,8 @@ public void testRestore() throws Exception { TimerInternals.TimerData.of( "timer1", nameSpace, - new Instant(10), - new Instant(10), + Instant.ofEpochMilli(10), + Instant.ofEpochMilli(10), TimeDomain.EVENT_TIME, CausedByDrain.NORMAL); timerInternals.setTimer(timer1); @@ -280,8 +280,8 @@ public void testRestore() throws Exception { TimerInternals.TimerData.of( "timer2", nameSpace, - new Instant(100), - new Instant(100), + Instant.ofEpochMilli(100), + Instant.ofEpochMilli(100), TimeDomain.EVENT_TIME, CausedByDrain.NORMAL); timerInternals.setTimer(timer2); @@ -293,7 +293,7 @@ public void testRestore() throws Exception { final SamzaTimerInternalsFactory restoredFactory = createTimerInternalsFactory(null, "timer", pipelineOptions, store); - restoredFactory.setInputWatermark(new Instant(150)); + restoredFactory.setInputWatermark(Instant.ofEpochMilli(150)); Collection> readyTimers = restoredFactory.removeReadyTimers(); assertEquals(2, readyTimers.size()); @@ -326,8 +326,8 @@ public void testProcessingTimeTimers() throws IOException { TimerInternals.TimerData.of( "timer1", nameSpace, - new Instant(10), - new Instant(10), + Instant.ofEpochMilli(10), + Instant.ofEpochMilli(10), TimeDomain.PROCESSING_TIME, CausedByDrain.NORMAL); timerInternals.setTimer(timer1); @@ -336,8 +336,8 @@ public void testProcessingTimeTimers() throws IOException { TimerInternals.TimerData.of( "timer2", nameSpace, - new Instant(100), - new Instant(100), + Instant.ofEpochMilli(100), + Instant.ofEpochMilli(100), TimeDomain.PROCESSING_TIME, CausedByDrain.NORMAL); timerInternals.setTimer(timer2); @@ -347,8 +347,8 @@ public void testProcessingTimeTimers() throws IOException { "timer3", "timerFamilyId3", nameSpace, - new Instant(100), - new Instant(100), + Instant.ofEpochMilli(100), + Instant.ofEpochMilli(100), TimeDomain.PROCESSING_TIME); timerInternals.setTimer(timer3); assertEquals(3, timerRegistry.timers.size()); @@ -387,8 +387,8 @@ public void testOverride() { TimerInternals.TimerData.of( "timerId", nameSpace, - new Instant(10), - new Instant(10), + Instant.ofEpochMilli(10), + Instant.ofEpochMilli(10), TimeDomain.EVENT_TIME, CausedByDrain.NORMAL); timerInternals.setTimer(timer1); @@ -398,8 +398,8 @@ public void testOverride() { TimerInternals.TimerData.of( "timerId", nameSpace, - new Instant(100), - new Instant(100), + Instant.ofEpochMilli(100), + Instant.ofEpochMilli(100), TimeDomain.EVENT_TIME, CausedByDrain.NORMAL); timerInternals.setTimer(timer2); @@ -408,22 +408,22 @@ public void testOverride() { TimerInternals.TimerData.of( "timerId2", nameSpace, - new Instant(200), - new Instant(200), + Instant.ofEpochMilli(200), + Instant.ofEpochMilli(200), TimeDomain.EVENT_TIME, CausedByDrain.NORMAL); timerInternals.setTimer(timer3); // this timer shouldn't override since it has a different id - timerInternalsFactory.setInputWatermark(new Instant(50)); + timerInternalsFactory.setInputWatermark(Instant.ofEpochMilli(50)); Collection> readyTimers = timerInternalsFactory.removeReadyTimers(); assertEquals(0, readyTimers.size()); - timerInternalsFactory.setInputWatermark(new Instant(150)); + timerInternalsFactory.setInputWatermark(Instant.ofEpochMilli(150)); readyTimers = timerInternalsFactory.removeReadyTimers(); assertEquals(1, readyTimers.size()); - timerInternalsFactory.setInputWatermark(new Instant(250)); + timerInternalsFactory.setInputWatermark(Instant.ofEpochMilli(250)); readyTimers = timerInternalsFactory.removeReadyTimers(); assertEquals(1, readyTimers.size()); @@ -468,8 +468,8 @@ private void testMaxExpiredEventTimersProcessAtOnce( TimerInternals.TimerData.of( "timer" + i, nameSpace, - new Instant(i), - new Instant(i), + Instant.ofEpochMilli(i), + Instant.ofEpochMilli(i), TimeDomain.EVENT_TIME, CausedByDrain.NORMAL); timerInternals.setTimer(timer); @@ -478,7 +478,7 @@ private void testMaxExpiredEventTimersProcessAtOnce( // Set the timestamp of the input watermark to be the value of totalNumberOfExpiredTimers // so that totalNumberOfExpiredTimers timers are expected be expired with respect to this // watermark. - final Instant inputWatermark = new Instant(totalNumberOfExpiredTimers); + final Instant inputWatermark = Instant.ofEpochMilli(totalNumberOfExpiredTimers); timerInternalsFactory.setInputWatermark(inputWatermark); final Collection> readyTimers = timerInternalsFactory.removeReadyTimers(); @@ -504,7 +504,12 @@ public void testBufferSizeNotExceedingPipelineOptionValue() { // timers in store are then timestamped from 0 - 4. for (int i = 0; i < 5; i++) { timerInternals.setTimer( - nameSpace, "timer" + i, "", new Instant(i), new Instant(i), TimeDomain.EVENT_TIME); + nameSpace, + "timer" + i, + "", + Instant.ofEpochMilli(i), + Instant.ofEpochMilli(i), + TimeDomain.EVENT_TIME); } // only two timers are supposed to be in event time buffer @@ -531,12 +536,17 @@ public void testAllTimersAreFiredWithReload() { // timers in store now are timestamped from 0 - 2. for (int i = 0; i < 3; i++) { timerInternals.setTimer( - nameSpace, "timer" + i, "", new Instant(i), new Instant(i), TimeDomain.EVENT_TIME); + nameSpace, + "timer" + i, + "", + Instant.ofEpochMilli(i), + Instant.ofEpochMilli(i), + TimeDomain.EVENT_TIME); } // total number of event time timers to fire equals to the number of timers in store Collection> readyTimers; - timerInternalsFactory.setInputWatermark(new Instant(3)); + timerInternalsFactory.setInputWatermark(Instant.ofEpochMilli(3)); readyTimers = timerInternalsFactory.removeReadyTimers(); // buffer should reload from store and all timers are supposed to be fired. assertEquals(3, readyTimers.size()); @@ -567,14 +577,19 @@ public void testAllTimersAreFiredInOrder() { // timers in store now are timestamped from 0 - 7. for (int i = 0; i < 8; i++) { timerInternals.setTimer( - nameSpace, "timer" + i, "", new Instant(i), new Instant(i), TimeDomain.EVENT_TIME); + nameSpace, + "timer" + i, + "", + Instant.ofEpochMilli(i), + Instant.ofEpochMilli(i), + TimeDomain.EVENT_TIME); } // fire the first 2 timers. // timers in memory now are timestamped from 2 - 4; // timers in store now are timestamped from 2 - 7. Collection> readyTimers; - timerInternalsFactory.setInputWatermark(new Instant(1)); + timerInternalsFactory.setInputWatermark(Instant.ofEpochMilli(1)); long lastTimestamp = 0; readyTimers = timerInternalsFactory.removeReadyTimers(); for (KeyedTimerData keyedTimerData : readyTimers) { @@ -591,9 +606,14 @@ public void testAllTimersAreFiredInOrder() { // the total number of timers to fire is 18. for (int i = 8; i < 20; i++) { timerInternals.setTimer( - nameSpace, "timer" + i, "", new Instant(i), new Instant(i), TimeDomain.EVENT_TIME); + nameSpace, + "timer" + i, + "", + Instant.ofEpochMilli(i), + Instant.ofEpochMilli(i), + TimeDomain.EVENT_TIME); } - timerInternalsFactory.setInputWatermark(new Instant(20)); + timerInternalsFactory.setInputWatermark(Instant.ofEpochMilli(20)); lastTimestamp = 0; readyTimers = timerInternalsFactory.removeReadyTimers(); for (KeyedTimerData keyedTimerData : readyTimers) { @@ -624,14 +644,19 @@ public void testNewTimersAreInsertedInOrder() { // timers in store now are timestamped from 0 - 9. for (int i = 0; i < 10; i++) { timerInternals.setTimer( - nameSpace, "timer" + i, "", new Instant(i), new Instant(i), TimeDomain.EVENT_TIME); + nameSpace, + "timer" + i, + "", + Instant.ofEpochMilli(i), + Instant.ofEpochMilli(i), + TimeDomain.EVENT_TIME); } // fire the first 2 timers. // timers in memory now are timestamped from 2 - 4; // timers in store now are timestamped from 2 - 9. Collection> readyTimers; - timerInternalsFactory.setInputWatermark(new Instant(1)); + timerInternalsFactory.setInputWatermark(Instant.ofEpochMilli(1)); long lastTimestamp = 0; readyTimers = timerInternalsFactory.removeReadyTimers(); for (KeyedTimerData keyedTimerData : readyTimers) { @@ -648,7 +673,12 @@ public void testNewTimersAreInsertedInOrder() { // prefixed with timer, timestamp is in order; for (int i = 0; i < 3; i++) { timerInternals.setTimer( - nameSpace, "timer" + i, "", new Instant(i), new Instant(i), TimeDomain.EVENT_TIME); + nameSpace, + "timer" + i, + "", + Instant.ofEpochMilli(i), + Instant.ofEpochMilli(i), + TimeDomain.EVENT_TIME); } // there are 11 timers in state now. @@ -656,7 +686,7 @@ public void testNewTimersAreInsertedInOrder() { // memory will be reloaded once to have 5 to 8 left (reload to have 4 to 8, but 4 is evicted), 5 // to 9 left in store. // all of them are in order for firing. - timerInternalsFactory.setInputWatermark(new Instant(5)); + timerInternalsFactory.setInputWatermark(Instant.ofEpochMilli(5)); lastTimestamp = 0; readyTimers = timerInternalsFactory.removeReadyTimers(); for (KeyedTimerData keyedTimerData : readyTimers) { @@ -668,7 +698,7 @@ public void testNewTimersAreInsertedInOrder() { assertEquals(4, timerInternalsFactory.getEventTimeBuffer().size()); // watermark 10 comes, so all timers will be evicted in order. - timerInternalsFactory.setInputWatermark(new Instant(10)); + timerInternalsFactory.setInputWatermark(Instant.ofEpochMilli(10)); readyTimers = timerInternalsFactory.removeReadyTimers(); for (KeyedTimerData keyedTimerData : readyTimers) { final long currentTimeStamp = keyedTimerData.getTimerData().getTimestamp().getMillis(); @@ -699,12 +729,17 @@ public void testBufferRefilledAfterRestoreToNonFullState() { // timer in store now is timestamped 6. for (int i = 0; i < 6; i++) { timerInternals.setTimer( - nameSpace, "timer" + i, "", new Instant(i), new Instant(i), TimeDomain.EVENT_TIME); + nameSpace, + "timer" + i, + "", + Instant.ofEpochMilli(i), + Instant.ofEpochMilli(i), + TimeDomain.EVENT_TIME); } // total number of event time timers to fire equals to the number of timers in store Collection> readyTimers; - timerInternalsFactory.setInputWatermark(new Instant(4)); + timerInternalsFactory.setInputWatermark(Instant.ofEpochMilli(4)); readyTimers = timerInternalsFactory.removeReadyTimers(); assertEquals(5, readyTimers.size()); // reloaded timer5 @@ -712,13 +747,18 @@ public void testBufferRefilledAfterRestoreToNonFullState() { for (int i = 6; i < 13; i++) { timerInternals.setTimer( - nameSpace, "timer" + i, "", new Instant(i), new Instant(i), TimeDomain.EVENT_TIME); + nameSpace, + "timer" + i, + "", + Instant.ofEpochMilli(i), + Instant.ofEpochMilli(i), + TimeDomain.EVENT_TIME); } // timers should go into buffer not state assertEquals(5, timerInternalsFactory.getEventTimeBuffer().size()); // watermark 10 comes,6 timers will be evicted in order and 2 still in buffer. - timerInternalsFactory.setInputWatermark(new Instant(10)); + timerInternalsFactory.setInputWatermark(Instant.ofEpochMilli(10)); readyTimers = timerInternalsFactory.removeReadyTimers(); long lastTimestamp = 0; for (KeyedTimerData keyedTimerData : readyTimers) { diff --git a/runners/samza/src/test/java/org/apache/beam/runners/samza/util/PipelineJsonRendererTest.java b/runners/samza/src/test/java/org/apache/beam/runners/samza/util/PipelineJsonRendererTest.java index 0a4f532808b1..948816f7f955 100644 --- a/runners/samza/src/test/java/org/apache/beam/runners/samza/util/PipelineJsonRendererTest.java +++ b/runners/samza/src/test/java/org/apache/beam/runners/samza/util/PipelineJsonRendererTest.java @@ -87,8 +87,8 @@ public void testCompositePipeline() throws IOException { p.apply( Create.timestamped( - TimestampedValue.of(KV.of(1, 1), new Instant(1)), - TimestampedValue.of(KV.of(2, 2), new Instant(2)))) + TimestampedValue.of(KV.of(1, 1), Instant.ofEpochMilli(1)), + TimestampedValue.of(KV.of(2, 2), Instant.ofEpochMilli(2)))) .apply(Window.into(FixedWindows.of(Duration.millis(10)))) .apply(Sum.integersPerKey()); diff --git a/runners/spark/3/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/CombineGloballyTest.java b/runners/spark/3/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/CombineGloballyTest.java index cca192df9de3..92485efd7e78 100644 --- a/runners/spark/3/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/CombineGloballyTest.java +++ b/runners/spark/3/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/CombineGloballyTest.java @@ -67,12 +67,12 @@ public void testCombineGloballyPreservesWindowing() { pipeline .apply( Create.timestamped( - TimestampedValue.of(1, new Instant(1)), - TimestampedValue.of(2, new Instant(2)), - TimestampedValue.of(3, new Instant(11)), - TimestampedValue.of(4, new Instant(3)), - TimestampedValue.of(5, new Instant(11)), - TimestampedValue.of(6, new Instant(12)))) + TimestampedValue.of(1, Instant.ofEpochMilli(1)), + TimestampedValue.of(2, Instant.ofEpochMilli(2)), + TimestampedValue.of(3, Instant.ofEpochMilli(11)), + TimestampedValue.of(4, Instant.ofEpochMilli(3)), + TimestampedValue.of(5, Instant.ofEpochMilli(11)), + TimestampedValue.of(6, Instant.ofEpochMilli(12)))) .apply(Window.into(FixedWindows.of(Duration.millis(10)))) .apply(Sum.integersGlobally().withoutDefaults()); PAssert.that(input).containsInAnyOrder(7, 14); @@ -85,12 +85,12 @@ public void testCombineGloballyWithSlidingWindows() { pipeline .apply( Create.timestamped( - TimestampedValue.of(1, new Instant(1)), - TimestampedValue.of(3, new Instant(2)), - TimestampedValue.of(5, new Instant(3)), - TimestampedValue.of(2, new Instant(1)), - TimestampedValue.of(4, new Instant(2)), - TimestampedValue.of(6, new Instant(3)))) + TimestampedValue.of(1, Instant.ofEpochMilli(1)), + TimestampedValue.of(3, Instant.ofEpochMilli(2)), + TimestampedValue.of(5, Instant.ofEpochMilli(3)), + TimestampedValue.of(2, Instant.ofEpochMilli(1)), + TimestampedValue.of(4, Instant.ofEpochMilli(2)), + TimestampedValue.of(6, Instant.ofEpochMilli(3)))) .apply(Window.into(SlidingWindows.of(Duration.millis(3)).every(Duration.millis(1)))) .apply(Sum.integersGlobally().withoutDefaults()); PAssert.that(input) @@ -104,9 +104,9 @@ public void testCombineGloballyWithMergingWindows() { pipeline .apply( Create.timestamped( - TimestampedValue.of(2, new Instant(5)), - TimestampedValue.of(4, new Instant(11)), - TimestampedValue.of(6, new Instant(12)))) + TimestampedValue.of(2, Instant.ofEpochMilli(5)), + TimestampedValue.of(4, Instant.ofEpochMilli(11)), + TimestampedValue.of(6, Instant.ofEpochMilli(12)))) .apply(Window.into(Sessions.withGapDuration(Duration.millis(5)))) .apply(Sum.integersGlobally().withoutDefaults()); @@ -120,9 +120,9 @@ public void testCountGloballyWithSlidingWindows() { pipeline .apply( Create.timestamped( - TimestampedValue.of("a", new Instant(1)), - TimestampedValue.of("a", new Instant(2)), - TimestampedValue.of("a", new Instant(2)))) + TimestampedValue.of("a", Instant.ofEpochMilli(1)), + TimestampedValue.of("a", Instant.ofEpochMilli(2)), + TimestampedValue.of("a", Instant.ofEpochMilli(2)))) .apply(Window.into(SlidingWindows.of(Duration.millis(2)).every(Duration.millis(1)))); PCollection output = input.apply(Combine.globally(Count.combineFn()).withoutDefaults()); @@ -136,9 +136,9 @@ public void testBinaryCombineWithSlidingWindows() { pipeline .apply( Create.timestamped( - TimestampedValue.of(1, new Instant(1)), - TimestampedValue.of(3, new Instant(2)), - TimestampedValue.of(5, new Instant(3)))) + TimestampedValue.of(1, Instant.ofEpochMilli(1)), + TimestampedValue.of(3, Instant.ofEpochMilli(2)), + TimestampedValue.of(5, Instant.ofEpochMilli(3)))) .apply(Window.into(SlidingWindows.of(Duration.millis(3)).every(Duration.millis(1)))) .apply( Combine.globally(BinaryCombineFn.of((i1, i2) -> i1 > i2 ? i1 : i2)) diff --git a/runners/spark/3/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/CombinePerKeyTest.java b/runners/spark/3/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/CombinePerKeyTest.java index 41c032cd85be..8c8c213155a6 100644 --- a/runners/spark/3/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/CombinePerKeyTest.java +++ b/runners/spark/3/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/CombinePerKeyTest.java @@ -89,12 +89,12 @@ public void testCombinePerKeyPreservesWindowing() { pipeline .apply( Create.timestamped( - TimestampedValue.of(KV.of(1, 1), new Instant(1)), - TimestampedValue.of(KV.of(1, 3), new Instant(2)), - TimestampedValue.of(KV.of(1, 5), new Instant(11)), - TimestampedValue.of(KV.of(2, 2), new Instant(3)), - TimestampedValue.of(KV.of(2, 4), new Instant(11)), - TimestampedValue.of(KV.of(2, 6), new Instant(12)))) + TimestampedValue.of(KV.of(1, 1), Instant.ofEpochMilli(1)), + TimestampedValue.of(KV.of(1, 3), Instant.ofEpochMilli(2)), + TimestampedValue.of(KV.of(1, 5), Instant.ofEpochMilli(11)), + TimestampedValue.of(KV.of(2, 2), Instant.ofEpochMilli(3)), + TimestampedValue.of(KV.of(2, 4), Instant.ofEpochMilli(11)), + TimestampedValue.of(KV.of(2, 6), Instant.ofEpochMilli(12)))) .apply(Window.into(FixedWindows.of(Duration.millis(10)))) .apply(Sum.integersPerKey()); PAssert.that(input).containsInAnyOrder(KV.of(1, 4), KV.of(1, 5), KV.of(2, 2), KV.of(2, 10)); @@ -107,12 +107,12 @@ public void testCombinePerKeyWithSlidingWindows() { pipeline .apply( Create.timestamped( - TimestampedValue.of(KV.of(1, 1), new Instant(1)), - TimestampedValue.of(KV.of(1, 3), new Instant(2)), - TimestampedValue.of(KV.of(1, 5), new Instant(3)), - TimestampedValue.of(KV.of(1, 2), new Instant(1)), - TimestampedValue.of(KV.of(1, 4), new Instant(2)), - TimestampedValue.of(KV.of(1, 6), new Instant(3)))) + TimestampedValue.of(KV.of(1, 1), Instant.ofEpochMilli(1)), + TimestampedValue.of(KV.of(1, 3), Instant.ofEpochMilli(2)), + TimestampedValue.of(KV.of(1, 5), Instant.ofEpochMilli(3)), + TimestampedValue.of(KV.of(1, 2), Instant.ofEpochMilli(1)), + TimestampedValue.of(KV.of(1, 4), Instant.ofEpochMilli(2)), + TimestampedValue.of(KV.of(1, 6), Instant.ofEpochMilli(3)))) .apply(Window.into(SlidingWindows.of(Duration.millis(3)).every(Duration.millis(1)))) .apply(Sum.integersPerKey()); PAssert.that(input) @@ -131,12 +131,12 @@ public void testCombineByKeyWithMergingWindows() { pipeline .apply( Create.timestamped( - TimestampedValue.of(KV.of(1, 1), new Instant(5)), - TimestampedValue.of(KV.of(1, 3), new Instant(7)), - TimestampedValue.of(KV.of(1, 5), new Instant(11)), - TimestampedValue.of(KV.of(2, 2), new Instant(5)), - TimestampedValue.of(KV.of(2, 4), new Instant(11)), - TimestampedValue.of(KV.of(2, 6), new Instant(12)))) + TimestampedValue.of(KV.of(1, 1), Instant.ofEpochMilli(5)), + TimestampedValue.of(KV.of(1, 3), Instant.ofEpochMilli(7)), + TimestampedValue.of(KV.of(1, 5), Instant.ofEpochMilli(11)), + TimestampedValue.of(KV.of(2, 2), Instant.ofEpochMilli(5)), + TimestampedValue.of(KV.of(2, 4), Instant.ofEpochMilli(11)), + TimestampedValue.of(KV.of(2, 6), Instant.ofEpochMilli(12)))) .apply(Window.into(Sessions.withGapDuration(Duration.millis(5)))) .apply(Sum.integersPerKey()); @@ -155,10 +155,10 @@ public void testCountPerElementWithSlidingWindows() { pipeline .apply( Create.timestamped( - TimestampedValue.of("a", new Instant(1)), - TimestampedValue.of("a", new Instant(2)), - TimestampedValue.of("b", new Instant(3)), - TimestampedValue.of("b", new Instant(4)))) + TimestampedValue.of("a", Instant.ofEpochMilli(1)), + TimestampedValue.of("a", Instant.ofEpochMilli(2)), + TimestampedValue.of("b", Instant.ofEpochMilli(3)), + TimestampedValue.of("b", Instant.ofEpochMilli(4)))) .apply(Window.into(SlidingWindows.of(Duration.millis(2)).every(Duration.millis(1)))); PCollection> output = input.apply(Count.perElement()); PAssert.that(output) diff --git a/runners/spark/3/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/GroupByKeyTest.java b/runners/spark/3/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/GroupByKeyTest.java index 6569b7b20cfc..b7e25e940a25 100644 --- a/runners/spark/3/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/GroupByKeyTest.java +++ b/runners/spark/3/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/GroupByKeyTest.java @@ -88,12 +88,12 @@ public void testGroupByKeyPreservesWindowing() { .apply( Create.timestamped( shuffleRandomly( - TimestampedValue.of(KV.of(1, 1), new Instant(1)), - TimestampedValue.of(KV.of(1, 3), new Instant(2)), - TimestampedValue.of(KV.of(1, 5), new Instant(11)), - TimestampedValue.of(KV.of(2, 2), new Instant(3)), - TimestampedValue.of(KV.of(2, 4), new Instant(11)), - TimestampedValue.of(KV.of(2, 6), new Instant(12))))) + TimestampedValue.of(KV.of(1, 1), Instant.ofEpochMilli(1)), + TimestampedValue.of(KV.of(1, 3), Instant.ofEpochMilli(2)), + TimestampedValue.of(KV.of(1, 5), Instant.ofEpochMilli(11)), + TimestampedValue.of(KV.of(2, 2), Instant.ofEpochMilli(3)), + TimestampedValue.of(KV.of(2, 4), Instant.ofEpochMilli(11)), + TimestampedValue.of(KV.of(2, 6), Instant.ofEpochMilli(12))))) .apply(Window.into(FixedWindows.of(Duration.millis(10)))) .apply(GroupByKey.create()) // Passert do not support multiple kv with same key (because multiple windows) @@ -114,12 +114,12 @@ public void testGroupByKeyExplodesMultipleWindows() { .apply( Create.timestamped( shuffleRandomly( - TimestampedValue.of(KV.of(1, 1), new Instant(5)), - TimestampedValue.of(KV.of(1, 3), new Instant(7)), - TimestampedValue.of(KV.of(1, 5), new Instant(11)), - TimestampedValue.of(KV.of(2, 2), new Instant(5)), - TimestampedValue.of(KV.of(2, 4), new Instant(11)), - TimestampedValue.of(KV.of(2, 6), new Instant(12))))) + TimestampedValue.of(KV.of(1, 1), Instant.ofEpochMilli(5)), + TimestampedValue.of(KV.of(1, 3), Instant.ofEpochMilli(7)), + TimestampedValue.of(KV.of(1, 5), Instant.ofEpochMilli(11)), + TimestampedValue.of(KV.of(2, 2), Instant.ofEpochMilli(5)), + TimestampedValue.of(KV.of(2, 4), Instant.ofEpochMilli(11)), + TimestampedValue.of(KV.of(2, 6), Instant.ofEpochMilli(12))))) .apply(Window.into(SlidingWindows.of(Duration.millis(10)).every(Duration.millis(5)))) .apply(GroupByKey.create()) // Passert do not support multiple kv with same key (because multiple windows) @@ -142,12 +142,12 @@ public void testGroupByKeyWithMergingWindows() { .apply( Create.timestamped( shuffleRandomly( - TimestampedValue.of(KV.of(1, 1), new Instant(5)), - TimestampedValue.of(KV.of(1, 3), new Instant(7)), - TimestampedValue.of(KV.of(1, 5), new Instant(11)), - TimestampedValue.of(KV.of(2, 2), new Instant(5)), - TimestampedValue.of(KV.of(2, 4), new Instant(11)), - TimestampedValue.of(KV.of(2, 6), new Instant(12))))) + TimestampedValue.of(KV.of(1, 1), Instant.ofEpochMilli(5)), + TimestampedValue.of(KV.of(1, 3), Instant.ofEpochMilli(7)), + TimestampedValue.of(KV.of(1, 5), Instant.ofEpochMilli(11)), + TimestampedValue.of(KV.of(2, 2), Instant.ofEpochMilli(5)), + TimestampedValue.of(KV.of(2, 4), Instant.ofEpochMilli(11)), + TimestampedValue.of(KV.of(2, 6), Instant.ofEpochMilli(12))))) .apply(Window.into(Sessions.withGapDuration(Duration.millis(5)))) .apply(GroupByKey.create()) // Passert do not support multiple kv with same key (because multiple windows) diff --git a/runners/spark/3/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/WindowAssignTest.java b/runners/spark/3/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/WindowAssignTest.java index ecb3e7ebdb5b..baad509f4a47 100644 --- a/runners/spark/3/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/WindowAssignTest.java +++ b/runners/spark/3/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/WindowAssignTest.java @@ -50,11 +50,11 @@ public void testWindowAssign() { pipeline .apply( Create.timestamped( - TimestampedValue.of(1, new Instant(1)), - TimestampedValue.of(2, new Instant(2)), - TimestampedValue.of(3, new Instant(3)), - TimestampedValue.of(4, new Instant(10)), - TimestampedValue.of(5, new Instant(11)))) + TimestampedValue.of(1, Instant.ofEpochMilli(1)), + TimestampedValue.of(2, Instant.ofEpochMilli(2)), + TimestampedValue.of(3, Instant.ofEpochMilli(3)), + TimestampedValue.of(4, Instant.ofEpochMilli(10)), + TimestampedValue.of(5, Instant.ofEpochMilli(11)))) .apply(Window.into(FixedWindows.of(Duration.millis(10)))) .apply(Sum.integersGlobally().withoutDefaults()); PAssert.that(input).containsInAnyOrder(6, 9); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java index 22e25e5272a2..16c12fd5ffab 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java @@ -139,7 +139,7 @@ private static void awaitWatermarksOrTimeout( .getMillis(); Long batchDurationMillis = testSparkPipelineOptions.getBatchIntervalMillis(); Instant stopPipelineWatermark = - new Instant(testSparkPipelineOptions.getStopPipelineWatermark()); + Instant.ofEpochMilli(testSparkPipelineOptions.getStopPipelineWatermark()); // we poll for pipeline status in batch-intervals. while this is not in-sync with Spark's // execution clock, this is good enough. // we break on timeout or end-of-time WM, which ever comes first. diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/CreateStream.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/CreateStream.java index 2609f75d1471..b396092feb5b 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/CreateStream.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/CreateStream.java @@ -123,7 +123,7 @@ private CreateStream( */ public static CreateStream of( Coder coder, Duration batchDuration, boolean forceWatermarkSync) { - return new CreateStream<>(batchDuration, new Instant(0), coder, forceWatermarkSync); + return new CreateStream<>(batchDuration, Instant.ofEpochMilli(0), coder, forceWatermarkSync); } /** diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SparkUnboundedSource.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SparkUnboundedSource.java index bea1557a7103..adfc0b845d80 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SparkUnboundedSource.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SparkUnboundedSource.java @@ -234,7 +234,7 @@ public scala.Option> compute(Time validTime) { new SparkWatermarks( globalLowWatermarkForBatch, globalHighWatermarkForBatch, - new Instant(validTime.milliseconds())); + Instant.ofEpochMilli(validTime.milliseconds())); // add to watermark queue. GlobalWatermarkHolder.add(inputDStreamId, sparkWatermark); } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkTimerInternals.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkTimerInternals.java index 9ef75635c212..6fc3912e7fcb 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkTimerInternals.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkTimerInternals.java @@ -63,7 +63,9 @@ public static SparkTimerInternals forStreamFromSources( || watermarks.isEmpty() || Collections.disjoint(sourceIds, watermarks.keySet())) { return new SparkTimerInternals( - BoundedWindow.TIMESTAMP_MIN_VALUE, BoundedWindow.TIMESTAMP_MIN_VALUE, new Instant(0)); + BoundedWindow.TIMESTAMP_MIN_VALUE, + BoundedWindow.TIMESTAMP_MIN_VALUE, + Instant.ofEpochMilli(0)); } // there might be more than one stream feeding this stream, slowest WM is the right one. Instant slowestLowWatermark = BoundedWindow.TIMESTAMP_MAX_VALUE; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkStreamingTranslationContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkStreamingTranslationContext.java index 7e20b2189c50..cfdc81845aab 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkStreamingTranslationContext.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkStreamingTranslationContext.java @@ -37,7 +37,7 @@ public SparkStreamingTranslationContext( super(jsc, options, jobInfo); Duration batchDuration = new Duration(options.getBatchIntervalMillis()); this.streamingContext = new JavaStreamingContext(jsc, batchDuration); - this.firstTimestamp = new Instant(); + this.firstTimestamp = Instant.now(); } public JavaStreamingContext getStreamingContext() { diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/TestDStream.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/TestDStream.java index abf2e7c3242e..c33515f2eb25 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/TestDStream.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/TestDStream.java @@ -108,7 +108,8 @@ private void waitForLastBatch(Time validTime) { private void addWatermark(Time time, WatermarkEvent event) { SparkWatermarks watermarks = - new SparkWatermarks(lastWatermark, event.getWatermark(), new Instant(time.milliseconds())); + new SparkWatermarks( + lastWatermark, event.getWatermark(), Instant.ofEpochMilli(time.milliseconds())); lastWatermark = event.getWatermark(); GlobalWatermarkHolder.add(id(), watermarks); } diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/GlobalWatermarkHolderTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/GlobalWatermarkHolderTest.java index a4dc6afd9c45..77f0c7e19442 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/GlobalWatermarkHolderTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/GlobalWatermarkHolderTest.java @@ -45,7 +45,7 @@ public class GlobalWatermarkHolderTest { @Test public void testLowHighWatermarksAdvance() { - Instant instant = new Instant(0); + Instant instant = Instant.ofEpochMilli(0); // low == high. GlobalWatermarkHolder.add( @@ -90,7 +90,7 @@ public void testLowHighWatermarksAdvance() { @Test public void testSynchronizedTimeMonotonic() { - Instant instant = new Instant(0); + Instant instant = Instant.ofEpochMilli(0); GlobalWatermarkHolder.add( 1, @@ -111,7 +111,7 @@ public void testSynchronizedTimeMonotonic() { @Test public void testMultiSource() { - Instant instant = new Instant(0); + Instant instant = Instant.ofEpochMilli(0); GlobalWatermarkHolder.add( 1, diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/SparkMetricsSinkTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/SparkMetricsSinkTest.java index 6c53ba8056ad..18016f2fc986 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/SparkMetricsSinkTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/SparkMetricsSinkTest.java @@ -89,7 +89,7 @@ public void testInStreamingMode() throws Exception { pipeline.getOptions().as(TestSparkPipelineOptions.class).setStreaming(true); assertThat(InMemoryMetrics.valueOf("emptyLines"), is(nullValue())); - Instant instant = new Instant(0); + Instant instant = Instant.ofEpochMilli(0); CreateStream source = CreateStream.of( StringUtf8Coder.of(), diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/metrics/SparkMetricsPusherTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/metrics/SparkMetricsPusherTest.java index dbd569d89d15..7f3d085abe90 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/metrics/SparkMetricsPusherTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/metrics/SparkMetricsPusherTest.java @@ -71,7 +71,7 @@ public void init() { public void testInStreamingMode() throws Exception { pipeline.getOptions().as(TestSparkPipelineOptions.class).setStreaming(true); - Instant instant = new Instant(0); + Instant instant = Instant.ofEpochMilli(0); CreateStream source = CreateStream.of(VarIntCoder.of(), batchDuration()) .emptyBatch() diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/AbstractInOutIteratorTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/AbstractInOutIteratorTest.java index cfda84c1981f..60459bd5c272 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/AbstractInOutIteratorTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/AbstractInOutIteratorTest.java @@ -54,8 +54,8 @@ public class AbstractInOutIteratorTest { private static final String TEST_KEY = "testKey"; private static final String TIMER_ID = "testTimerId"; private static final String TIMER_FAMILY_ID = "testTimerFamilyId"; - private static final Instant TEST_TIMESTAMP = new Instant(42L); - private static final Instant TEST_OUTPUT_TIMESTAMP = new Instant(84L); + private static final Instant TEST_TIMESTAMP = Instant.ofEpochMilli(42L); + private static final Instant TEST_OUTPUT_TIMESTAMP = Instant.ofEpochMilli(84L); private static final TimeDomain TEST_TIME_DOMAIN = TimeDomain.EVENT_TIME; private static final CausedByDrain TEST_CAUSED_BY_DRAIN = CausedByDrain.CAUSED_BY_DRAIN; diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/CreateStreamTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/CreateStreamTest.java index 243f3a3e533f..c1d7c27a98ef 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/CreateStreamTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/CreateStreamTest.java @@ -91,7 +91,7 @@ public class CreateStreamTest implements Serializable { @Test public void testLateDataAccumulating() throws IOException { - Instant instant = new Instant(0); + Instant instant = Instant.ofEpochMilli(0); CreateStream source = CreateStream.of(VarIntCoder.of(), batchDuration()) .emptyBatch() @@ -163,14 +163,14 @@ public void testDiscardingMode() throws IOException { CreateStream source = CreateStream.of(StringUtf8Coder.of(), batchDuration()) .nextBatch( - TimestampedValue.of("firstPane", new Instant(100)), - TimestampedValue.of("alsoFirstPane", new Instant(200))) - .advanceWatermarkForNextBatch(new Instant(1001L)) - .nextBatch(TimestampedValue.of("onTimePane", new Instant(500))) + TimestampedValue.of("firstPane", Instant.ofEpochMilli(100)), + TimestampedValue.of("alsoFirstPane", Instant.ofEpochMilli(200))) + .advanceWatermarkForNextBatch(Instant.ofEpochMilli(1001L)) + .nextBatch(TimestampedValue.of("onTimePane", Instant.ofEpochMilli(500))) .advanceNextBatchWatermarkToInfinity() .nextBatch( - TimestampedValue.of("finalLatePane", new Instant(750)), - TimestampedValue.of("alsoFinalLatePane", new Instant(250))); + TimestampedValue.of("finalLatePane", Instant.ofEpochMilli(750)), + TimestampedValue.of("alsoFinalLatePane", Instant.ofEpochMilli(250))); FixedWindows windowFn = FixedWindows.of(Duration.millis(1000L)); Duration allowedLateness = Duration.millis(5000L); @@ -189,7 +189,7 @@ public void testDiscardingMode() throws IOException { .apply(Values.create()) .apply(Flatten.iterables()); - IntervalWindow window = windowFn.assignWindow(new Instant(100)); + IntervalWindow window = windowFn.assignWindow(Instant.ofEpochMilli(100)); PAssert.that(values) .inWindow(window) .containsInAnyOrder( @@ -207,15 +207,15 @@ public void testDiscardingMode() throws IOException { @Test public void testFirstElementLate() throws IOException { - Instant lateElementTimestamp = new Instant(-1_000_000); + Instant lateElementTimestamp = Instant.ofEpochMilli(-1_000_000); CreateStream source = CreateStream.of(StringUtf8Coder.of(), batchDuration()) .emptyBatch() - .advanceWatermarkForNextBatch(new Instant(0)) + .advanceWatermarkForNextBatch(Instant.ofEpochMilli(0)) .emptyBatch() .nextBatch( TimestampedValue.of("late", lateElementTimestamp), - TimestampedValue.of("onTime", new Instant(100))) + TimestampedValue.of("onTime", Instant.ofEpochMilli(100))) .advanceNextBatchWatermarkToInfinity(); FixedWindows windowFn = FixedWindows.of(Duration.millis(1000L)); @@ -234,7 +234,7 @@ public void testFirstElementLate() throws IOException { PAssert.that(values).inWindow(windowFn.assignWindow(lateElementTimestamp)).empty(); PAssert.that(values) - .inWindow(windowFn.assignWindow(new Instant(100))) + .inWindow(windowFn.assignWindow(Instant.ofEpochMilli(100))) .containsInAnyOrder("onTime"); p.run(); @@ -301,7 +301,7 @@ public void testMultipleStreams() throws IOException { @Test public void testFlattenedWithWatermarkHold() throws IOException { - Instant instant = new Instant(0); + Instant instant = Instant.ofEpochMilli(0); CreateStream source1 = CreateStream.of(VarIntCoder.of(), batchDuration()) .emptyBatch() @@ -362,7 +362,7 @@ public void testFlattenedWithWatermarkHold() throws IOException { */ @Test public void testMultiOutputParDo() throws IOException { - Instant instant = new Instant(0); + Instant instant = Instant.ofEpochMilli(0); CreateStream source1 = CreateStream.of(VarIntCoder.of(), batchDuration()) .emptyBatch() @@ -408,7 +408,7 @@ public void process(ProcessContext context) { */ @Test public void testParDoCallsSetupAndTeardown() { - Instant instant = new Instant(0); + Instant instant = Instant.ofEpochMilli(0); p.apply( CreateStream.of(VarIntCoder.of(), batchDuration()) @@ -445,9 +445,11 @@ public void testElementAtPositiveInfinityThrows() { public void testAdvanceWatermarkNonMonotonicThrows() { CreateStream source = CreateStream.of(VarIntCoder.of(), batchDuration()) - .advanceWatermarkForNextBatch(new Instant(0L)); + .advanceWatermarkForNextBatch(Instant.ofEpochMilli(0L)); thrown.expect(IllegalArgumentException.class); - source.advanceWatermarkForNextBatch(new Instant(-1L)).advanceNextBatchWatermarkToInfinity(); + source + .advanceWatermarkForNextBatch(Instant.ofEpochMilli(-1L)) + .advanceNextBatchWatermarkToInfinity(); } @Test @@ -462,7 +464,7 @@ public void testAdvanceWatermarkEqualToPositiveInfinityThrows() { @Test public void testInStreamingModeCountByKey() throws Exception { - Instant instant = new Instant(0); + Instant instant = Instant.ofEpochMilli(0); CreateStream> kvSource = CreateStream.of(KvCoder.of(VarIntCoder.of(), VarLongCoder.of()), batchDuration()) diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/ResumeFromCheckpointStreamingTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/ResumeFromCheckpointStreamingTest.java index 334a0c832b3a..f730fd575042 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/ResumeFromCheckpointStreamingTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/ResumeFromCheckpointStreamingTest.java @@ -150,10 +150,10 @@ public void testWithResume() throws Exception { // write to Kafka produce( ImmutableMap.of( - "k1", new Instant(100), - "k2", new Instant(200), - "k3", new Instant(300), - "k4", new Instant(400))); + "k1", Instant.ofEpochMilli(100), + "k2", Instant.ofEpochMilli(200), + "k3", Instant.ofEpochMilli(300), + "k4", Instant.ofEpochMilli(400))); MetricsFilter metricsFilter = MetricsFilter.builder() @@ -161,7 +161,7 @@ public void testWithResume() throws Exception { .build(); // first run should expect EOT matching the last injected element. - SparkPipelineResult res = run(Optional.of(new Instant(400)), 0); + SparkPipelineResult res = run(Optional.of(Instant.ofEpochMilli(400)), 0); assertThat( res.metrics().queryMetrics(metricsFilter).getCounters(), @@ -188,8 +188,8 @@ public void testWithResume() throws Exception { // - write a bit more. produce( ImmutableMap.of( - "k5", new Instant(499), - "EOF", new Instant(500) // to be dropped from [0, 500). + "k5", Instant.ofEpochMilli(499), + "EOF", Instant.ofEpochMilli(500) // to be dropped from [0, 500). )); // recovery should resume from last read offset, and read the second batch of input. diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/SparkCoGroupByKeyStreamingTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/SparkCoGroupByKeyStreamingTest.java index b64fa8e33eb9..33498aa0397a 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/SparkCoGroupByKeyStreamingTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/SparkCoGroupByKeyStreamingTest.java @@ -64,7 +64,7 @@ private Duration batchDuration() { @Category(StreamingTest.class) @Test public void testInStreamingMode() throws Exception { - Instant instant = new Instant(0); + Instant instant = Instant.ofEpochMilli(0); CreateStream> source1 = CreateStream.of(KvCoder.of(VarIntCoder.of(), VarIntCoder.of()), batchDuration()) .emptyBatch() diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/StatefulStreamingParDoEvaluatorTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/StatefulStreamingParDoEvaluatorTest.java index d16d295f7af4..3527694b3f0c 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/StatefulStreamingParDoEvaluatorTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/StatefulStreamingParDoEvaluatorTest.java @@ -69,7 +69,7 @@ public class StatefulStreamingParDoEvaluatorTest implements Serializable { private PTransform>> createStreamingSource( Pipeline pipeline) { - Instant instant = new Instant(0); + Instant instant = Instant.ofEpochMilli(0); final KvCoder coder = KvCoder.of(VarIntCoder.of(), VarIntCoder.of()); final Duration batchDuration = batchDuration(pipeline); return CreateStream.of(coder, batchDuration) @@ -89,7 +89,7 @@ private PTransform>> createStreamingSou private PTransform>> createStreamingSource( Pipeline pipeline, int iterCount) { - Instant instant = new Instant(0); + Instant instant = Instant.ofEpochMilli(0); final KvCoder coder = KvCoder.of(VarIntCoder.of(), VarIntCoder.of()); final Duration batchDuration = batchDuration(pipeline); @@ -273,7 +273,7 @@ public void shouldTriggerProcessingTimeTimer() { public void shouldTriggerProcessingTimeTimerWithSparseKey() { final int sparseKey = 3; KvCoder coder = KvCoder.of(VarIntCoder.of(), VarIntCoder.of()); - Instant instant = new Instant(0); + Instant instant = Instant.ofEpochMilli(0); CreateStream> sparseStream = CreateStream.of(coder, batchDuration(p)).advanceWatermarkForNextBatch(instant); diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslatorTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslatorTest.java index 1b607be23ded..74d3b4b53846 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslatorTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslatorTest.java @@ -129,7 +129,7 @@ public PCollectionView> expand(PBegin input) { public void testStreamingSideInputAsIterableView() { final PipelineFunction pipelineFunction = (PipelineOptions options) -> { - final Instant baseTimestamp = new Instant(0); + final Instant baseTimestamp = Instant.ofEpochMilli(0); final Pipeline p = Pipeline.create(options); final PCollectionView> streamingSideInput = @@ -150,7 +150,8 @@ public void testStreamingSideInputAsIterableView() { return p; }; - final PipelineResult result = run(pipelineFunction, Optional.of(new Instant(1000)), true); + final PipelineResult result = + run(pipelineFunction, Optional.of(Instant.ofEpochMilli(1000)), true); final Iterable> distributions = result .metrics() @@ -216,7 +217,7 @@ public void testFlattenPCollResumeFromCheckpoint() { final PipelineFunction pipelineFunction = (PipelineOptions options) -> { Pipeline p = Pipeline.create(options); - final Instant baseTimestamp = new Instant(0); + final Instant baseTimestamp = Instant.ofEpochMilli(0); final PCollection bounded = p.apply( "Bounded", @@ -238,7 +239,7 @@ public void testFlattenPCollResumeFromCheckpoint() { return p; }; - PipelineResult res = run(pipelineFunction, Optional.of(new Instant(400)), false); + PipelineResult res = run(pipelineFunction, Optional.of(Instant.ofEpochMilli(400)), false); // Verify metrics for Bounded PCollection (sum of 0-9 = 45, count = 10) assertThat( diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/util/TimerUtilsTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/util/TimerUtilsTest.java index a91b92aefadb..ba4cae9e3e11 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/util/TimerUtilsTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/util/TimerUtilsTest.java @@ -49,7 +49,7 @@ public class TimerUtilsTest { @Mock private TimerInternals.TimerData activeTimer; @Mock private IntervalWindow mockWindow; - private static final Instant NOW = new Instant(1000L); + private static final Instant NOW = Instant.ofEpochMilli(1000L); private static final Duration ALLOWED_LATENESS = Duration.standardMinutes(5); @Before 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 71749566a2f0..da40461ab23c 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 @@ -74,7 +74,7 @@ public Instant decode(InputStream inStream) throws CoderException, IOException { // // 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 - return new Instant(shiftedMillis + Long.MIN_VALUE); + return Instant.ofEpochMilli(shiftedMillis + Long.MIN_VALUE); } @Override diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/MatchResult.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/MatchResult.java index 8a181d765ce1..810d6cc7f783 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/MatchResult.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/MatchResult.java @@ -107,7 +107,7 @@ public abstract static class Metadata implements Serializable { * PCollection metadataWithTimestamp = p * .apply(FileIO.match().filepattern("hdfs://path/to/*.gz")) * .setCoder(MetadataCoderV2.of()) - * .apply(WithTimestamps.of(metadata -> new Instant(metadata.lastModifiedMillis()))); + * .apply(WithTimestamps.of(metadata -> Instant.ofEpochMilli(metadata.lastModifiedMillis()))); * } */ public abstract long lastModifiedMillis(); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/GaugeResult.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/GaugeResult.java index da3268b003d4..e185279ec0b7 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/GaugeResult.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/GaugeResult.java @@ -39,7 +39,7 @@ public static GaugeResult empty() { public static class EmptyGaugeResult extends GaugeResult { private static final EmptyGaugeResult INSTANCE = new EmptyGaugeResult(); - private static final Instant EPOCH = new Instant(0); + private static final Instant EPOCH = Instant.ofEpochMilli(0); private EmptyGaugeResult() {} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ByteBuddyUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ByteBuddyUtils.java index 8bc6c99ca5c6..0fab67ea3c46 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ByteBuddyUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ByteBuddyUtils.java @@ -849,10 +849,10 @@ protected StackManipulation convertDateTime(TypeDescriptor type) { // Otherwise, generate the following code: // // for ReadableInstant: - // return new Instant(value.getMillis()); + // return Instant.ofEpochMilli(value.getMillis()); // // for ReadablePartial: - // return new Instant((value.toDateTime(Instant.EPOCH)).getMillis()); + // return Instant.ofEpochMilli((value.toDateTime(Instant.EPOCH)).getMillis()); List stackManipulations = new ArrayList<>(); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/WindowFnTestUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/WindowFnTestUtils.java index c77c10c8faaf..7f5cd5eeb49e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/WindowFnTestUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/WindowFnTestUtils.java @@ -60,7 +60,7 @@ public static Map> runWindowFn( WindowFn windowFn, List timestamps) throws Exception { List> timestampedValues = new ArrayList<>(); for (Long timestamp : timestamps) { - timestampedValues.add(TimestampedValue.of((T) null, new Instant(timestamp))); + timestampedValues.add(TimestampedValue.of((T) null, Instant.ofEpochMilli(timestamp))); } return runWindowFnWithValue(windowFn, timestampedValues); } @@ -92,7 +92,7 @@ public static Map> runWindowFnWithVa public static Collection assignedWindows( WindowFn windowFn, long timestamp) throws Exception { return assignedWindowsWithValue( - windowFn, TimestampedValue.of((T) null, new Instant(timestamp))); + windowFn, TimestampedValue.of((T) null, Instant.ofEpochMilli(timestamp))); } /** @@ -106,7 +106,7 @@ public static Collection assignedWindowsWithValu } private static String timestampValue(long timestamp) { - return "T" + new Instant(timestamp); + return "T" + Instant.ofEpochMilli(timestamp); } /** Test implementation of AssignContext. */ @@ -219,7 +219,7 @@ public static void validateGetOutputTimestamps( for (List timestamps : timestampsPerWindow) { List> timestampedValues = new ArrayList<>(); for (Long timestamp : timestamps) { - TimestampedValue tv = TimestampedValue.of(null, new Instant(timestamp)); + TimestampedValue tv = TimestampedValue.of(null, Instant.ofEpochMilli(timestamp)); timestampedValues.add(tv); } timestampValuesPerWindow.add(timestampedValues); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java index a2f32b8b3dd3..433484b21eb3 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java @@ -295,7 +295,7 @@ public static TimestampedValues timestamped( Iterator valueIter = values.iterator(); Iterator timestampIter = timestamps.iterator(); while (valueIter.hasNext() && timestampIter.hasNext()) { - elems.add(TimestampedValue.of(valueIter.next(), new Instant(timestampIter.next()))); + elems.add(TimestampedValue.of(valueIter.next(), Instant.ofEpochMilli(timestampIter.next()))); } checkArgument( !valueIter.hasNext() && !timestampIter.hasNext(), diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTime.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTime.java index c9612c069207..7c20c2a51a6e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTime.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTime.java @@ -91,10 +91,10 @@ public AfterProcessingTime alignedTo(final Duration period, final Instant offset /** * Aligns the time to be the smallest multiple of {@code period} greater than the epoch boundary - * (aka {@code new Instant(0)}). + * (aka {@code Instant.ofEpochMilli(0)}). */ public AfterProcessingTime alignedTo(final Duration period) { - return alignedTo(period, new Instant(0)); + return alignedTo(period, Instant.ofEpochMilli(0)); } @Override diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java index 217461dbe0a9..bba603095643 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/BoundedWindow.java @@ -79,7 +79,7 @@ public static String formatTimestamp(Instant timestamp) { /** Parses a timestamp from the proto. */ private static Instant extractTimestampFromProto(RunnerApi.BeamConstants.Constants constant) { - return new Instant( + return Instant.ofEpochMilli( Long.parseLong( constant.getValueDescriptor().getOptions().getExtension(RunnerApi.beamConstant))); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/FixedWindows.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/FixedWindows.java index 851f606aee2a..1adaa7795e37 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/FixedWindows.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/FixedWindows.java @@ -73,7 +73,7 @@ private FixedWindows(Duration size, Duration offset) { @Override public IntervalWindow assignWindow(Instant timestamp) { Instant start = - new Instant( + Instant.ofEpochMilli( timestamp.getMillis() - timestamp.plus(size).minus(offset).getMillis() % size.getMillis()); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindow.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindow.java index cadb2b33bb2a..f37355da6ac2 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindow.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindow.java @@ -91,7 +91,7 @@ private Coder() {} /** Parses the max timestamp for global windows from the proto. */ private static Instant extractMaxTimestampFromProto() { - return new Instant( + return Instant.ofEpochMilli( Long.parseLong( RunnerApi.BeamConstants.Constants.GLOBAL_WINDOW_MAX_TIMESTAMP_MILLIS .getValueDescriptor() diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/IntervalWindow.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/IntervalWindow.java index 99382c60ce11..58b31ecb9c1e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/IntervalWindow.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/IntervalWindow.java @@ -96,8 +96,8 @@ public boolean intersects(IntervalWindow other) { /** Returns the minimal window that includes both this window and the given window. */ public IntervalWindow span(IntervalWindow other) { return new IntervalWindow( - new Instant(Math.min(start.getMillis(), other.start.getMillis())), - new Instant(Math.max(end.getMillis(), other.end.getMillis()))); + Instant.ofEpochMilli(Math.min(start.getMillis(), other.start.getMillis())), + Instant.ofEpochMilli(Math.max(end.getMillis(), other.end.getMillis()))); } @Override diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/SlidingWindows.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/SlidingWindows.java index 5bbc2c9a720a..76ff617a54e1 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/SlidingWindows.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/SlidingWindows.java @@ -111,7 +111,7 @@ public Collection assignWindows(Instant timestamp) { for (long start = lastStart; start > timestamp.minus(size).getMillis(); start -= period.getMillis()) { - windows.add(new IntervalWindow(new Instant(start), size)); + windows.add(new IntervalWindow(Instant.ofEpochMilli(start), size)); } return windows; } @@ -130,7 +130,7 @@ public IntervalWindow getSideInputWindow(BoundedWindow mainWindow) { "Attempted to get side input window for GlobalWindow from non-global WindowFn"); } long lastStart = lastStartFor(mainWindow.maxTimestamp().minus(size)); - return new IntervalWindow(new Instant(lastStart + period.getMillis()), size); + return new IntervalWindow(Instant.ofEpochMilli(lastStart + period.getMillis()), size); } }; } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/TimestampTransform.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/TimestampTransform.java index c0529b94c181..3da202f0f899 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/TimestampTransform.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/TimestampTransform.java @@ -62,7 +62,7 @@ public static TimestampTransform alignTo(Duration period, Instant offset) { */ @Internal public static TimestampTransform alignTo(Duration period) { - return alignTo(period, new Instant(0)); + return alignTo(period, Instant.ofEpochMilli(0)); } /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/TestStreamTranslation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/TestStreamTranslation.java index db1a2f875c90..64d1a2aa5b6f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/TestStreamTranslation.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/TestStreamTranslation.java @@ -139,7 +139,7 @@ static TestStream.Event eventFromProto( switch (protoEvent.getEventCase()) { case WATERMARK_EVENT: return TestStream.WatermarkEvent.advanceTo( - new Instant(protoEvent.getWatermarkEvent().getNewWatermark())); + Instant.ofEpochMilli(protoEvent.getWatermarkEvent().getNewWatermark())); case PROCESSING_TIME_EVENT: return TestStream.ProcessingTimeEvent.advanceBy( Duration.millis(protoEvent.getProcessingTimeEvent().getAdvanceDuration())); @@ -150,7 +150,7 @@ static TestStream.Event eventFromProto( decodedElements.add( TimestampedValue.of( CoderUtils.decodeFromByteArray(coder, element.getEncodedElement().toByteArray()), - new Instant(element.getTimestamp()))); + Instant.ofEpochMilli(element.getTimestamp()))); } return TestStream.ElementEvent.add(decodedElements); case EVENT_NOT_SET: diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/TriggerTranslation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/TriggerTranslation.java index d4ead4ce8946..b1f79bd5c1b5 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/TriggerTranslation.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/TriggerTranslation.java @@ -281,7 +281,7 @@ public static Trigger fromProto(RunnerApi.Trigger triggerProto) { trigger = trigger.alignedTo( Duration.millis(transform.getAlignTo().getPeriod()), - new Instant(transform.getAlignTo().getOffset())); + Instant.ofEpochMilli(transform.getAlignTo().getOffset())); break; case DELAY: trigger = trigger.plusDelayOf(Duration.millis(transform.getDelay().getDelayMillis())); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/InstantCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/InstantCoderTest.java index 4daff2b5557b..8ed669bf76af 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/InstantCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/InstantCoderTest.java @@ -51,7 +51,7 @@ public class InstantCoderTest { static { TEST_VALUES = Lists.newArrayList(); for (long timestamp : TEST_TIMESTAMPS) { - TEST_VALUES.add(new Instant(timestamp)); + TEST_VALUES.add(Instant.ofEpochMilli(timestamp)); } } @@ -69,7 +69,7 @@ public void testOrderedEncoding() throws Exception { List encodings = new ArrayList<>(sortedTimestamps.size()); for (long timestamp : sortedTimestamps) { - encodings.add(CoderUtils.encodeToByteArray(TEST_CODER, new Instant(timestamp))); + encodings.add(CoderUtils.encodeToByteArray(TEST_CODER, Instant.ofEpochMilli(timestamp))); } // Verify that the encodings were already sorted, since they were generated diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/RowCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/RowCoderTest.java index 885ff8f1491a..cf8d9d1c97f7 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/RowCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/RowCoderTest.java @@ -65,7 +65,7 @@ public void testPrimitiveTypes() throws Exception { .build(); DateTime dateTime = - new DateTime().withDate(1979, 3, 14).withTime(1, 2, 3, 4).withZone(DateTimeZone.UTC); + DateTime.now().withDate(1979, 3, 14).withTime(1, 2, 3, 4).withZone(DateTimeZone.UTC); Row row = Row.withSchema(schema) .addValues( diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/TimestampPrefixingWindowCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/TimestampPrefixingWindowCoderTest.java index 6f71962ecc7e..a0aa9f7c68f8 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/TimestampPrefixingWindowCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/TimestampPrefixingWindowCoderTest.java @@ -119,17 +119,17 @@ public void registerByteSizeObserver(CustomWindow value, ElementByteSizeObserver private static final List CUSTOM_WINDOW_LIST = Lists.newArrayList( - new CustomWindow(new Instant(0L), new Instant(1L), true), - new CustomWindow(new Instant(100L), new Instant(200L), false), - new CustomWindow(new Instant(0L), BoundedWindow.TIMESTAMP_MAX_VALUE, true)); + new CustomWindow(Instant.ofEpochMilli(0L), Instant.ofEpochMilli(1L), true), + new CustomWindow(Instant.ofEpochMilli(100L), Instant.ofEpochMilli(200L), false), + new CustomWindow(Instant.ofEpochMilli(0L), BoundedWindow.TIMESTAMP_MAX_VALUE, true)); @Test public void testEncodeAndDecode() throws Exception { List intervalWindowsToTest = Lists.newArrayList( - new IntervalWindow(new Instant(0L), new Instant(1L)), - new IntervalWindow(new Instant(100L), new Instant(200L)), - new IntervalWindow(new Instant(0L), BoundedWindow.TIMESTAMP_MAX_VALUE)); + new IntervalWindow(Instant.ofEpochMilli(0L), Instant.ofEpochMilli(1L)), + new IntervalWindow(Instant.ofEpochMilli(100L), Instant.ofEpochMilli(200L)), + new IntervalWindow(Instant.ofEpochMilli(0L), BoundedWindow.TIMESTAMP_MAX_VALUE)); TimestampPrefixingWindowCoder coder1 = TimestampPrefixingWindowCoder.of(IntervalWindow.getCoder()); for (IntervalWindow window : intervalWindowsToTest) { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSourceTest.java index a9879746f8e5..6001b7715243 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSourceTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSourceTest.java @@ -287,7 +287,7 @@ public KV getCurrent() { @Override public Instant getCurrentTimestamp() { - return new Instant(current); + return Instant.ofEpochMilli(current); } @Override @@ -311,7 +311,7 @@ public TestCountingSource getCurrentSource() { public Instant getWatermark() { // The watermark is a promise about future elements, and the timestamps of elements are // strictly increasing for this source. - return new Instant(current + 1); + return Instant.ofEpochMilli(current + 1); } @Override diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CompressedSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CompressedSourceTest.java index bb2fb09577f7..3c1b58605e55 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CompressedSourceTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CompressedSourceTest.java @@ -1143,7 +1143,7 @@ protected long getCurrentOffset() { @Override public Instant getCurrentTimestamp() throws NoSuchElementException { - return new Instant(getCurrentOffset()); + return Instant.ofEpochMilli(getCurrentOffset()); } } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingSourceTest.java index 70a09083619d..4641dedfa1c6 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingSourceTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingSourceTest.java @@ -284,7 +284,7 @@ public void testUnboundedSourceRateSplits() throws Exception { private static class ValueAsTimestampFn implements SerializableFunction { @Override public Instant apply(Long input) { - return new Instant(input); + return Instant.ofEpochMilli(input); } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/GenerateSequenceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/GenerateSequenceTest.java index 3907141b0549..035e906d9d5a 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/GenerateSequenceTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/GenerateSequenceTest.java @@ -200,7 +200,7 @@ public void testBuildExternal() { private static class ValueAsTimestampFn implements SerializableFunction { @Override public Instant apply(Long input) { - return new Instant(input); + return Instant.ofEpochMilli(input); } } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java index cc174002bb46..de62c2896c6e 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java @@ -378,7 +378,7 @@ public void testWithRunnerDeterminedShardingTestStream() throws IOException { for (int i = 0; i < 30; ++i) { elements.add("number: " + i); } - Instant startInstant = new Instant(0L); + Instant startInstant = Instant.ofEpochMilli(0L); TestStream testStream = TestStream.create(StringUtf8Coder.of()) // Initialize watermark for timer to be triggered correctly. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaCoderTest.java index cc57d382af0c..87239f2da921 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaCoderTest.java @@ -235,13 +235,13 @@ public static Collection data() throws NoSuchSchemaException { "foo", 9001, 0L, - new DateTime().withDate(1979, 3, 14).withTime(10, 30, 0, 0)), + DateTime.now().withDate(1979, 3, 14).withTime(10, 30, 0, 0)), () -> SimpleAutoValue.of( "bar", 9002, 1L, - new DateTime().withDate(1989, 3, 14).withTime(10, 30, 0, 0))), + DateTime.now().withDate(1989, 3, 14).withTime(10, 30, 0, 0))), true }, new Object[] { @@ -252,13 +252,13 @@ public static Collection data() throws NoSuchSchemaException { "foo", 9001, 0L, - new DateTime().withDate(1979, 3, 14).withTime(10, 30, 0, 0)), + DateTime.now().withDate(1979, 3, 14).withTime(10, 30, 0, 0)), () -> new SimpleBean( "bar", 9002, 1L, - new DateTime().withDate(1989, 3, 14).withTime(10, 30, 0, 0))), + DateTime.now().withDate(1989, 3, 14).withTime(10, 30, 0, 0))), true }, new Object[] { @@ -269,13 +269,13 @@ public static Collection data() throws NoSuchSchemaException { "foo", 9001, 0L, - new DateTime().withDate(1979, 3, 14).withTime(10, 30, 0, 0)), + DateTime.now().withDate(1979, 3, 14).withTime(10, 30, 0, 0)), () -> new SimplePojo( "bar", 9002, 1L, - new DateTime().withDate(1989, 3, 14).withTime(10, 30, 0, 0))), + DateTime.now().withDate(1989, 3, 14).withTime(10, 30, 0, 0))), true }, new Object[] { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaTranslationTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaTranslationTest.java index fb9a8308fcff..120e155711fc 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaTranslationTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaTranslationTest.java @@ -384,7 +384,7 @@ public static Iterable data() { .add(simpleRow(FieldType.map(FieldType.STRING, FieldType.INT32), map)) .add(simpleRow(FieldType.array(FieldType.STRING), list)) .add(simpleRow(FieldType.row(row.getSchema()), row)) - .add(simpleRow(FieldType.DATETIME, new Instant(23L))) + .add(simpleRow(FieldType.DATETIME, Instant.ofEpochMilli(23L))) .add(simpleRow(FieldType.DECIMAL, BigDecimal.valueOf(100000))) .add(simpleRow(FieldType.logicalType(new PortableNullArgLogicalType()), "str")) .add(simpleRow(FieldType.logicalType(new DateTime()), LocalDateTime.of(2000, 1, 3, 3, 1))) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/state/StateContextsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/state/StateContextsTest.java index 1b7112c40da2..25183baab41c 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/state/StateContextsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/state/StateContextsTest.java @@ -71,7 +71,7 @@ public void nullContextThrowsOnOptions() { @Test public void windowOnlyContextThrowsOnOptions() { - BoundedWindow window = new IntervalWindow(new Instant(-137), Duration.millis(21L)); + BoundedWindow window = new IntervalWindow(Instant.ofEpochMilli(-137), Duration.millis(21L)); StateContext context = StateContexts.windowOnlyContext(window); thrown.expect(IllegalArgumentException.class); context.getPipelineOptions(); @@ -79,7 +79,7 @@ public void windowOnlyContextThrowsOnOptions() { @Test public void windowOnlyContextThrowsOnSideInput() { - BoundedWindow window = new IntervalWindow(new Instant(-137), Duration.millis(21L)); + BoundedWindow window = new IntervalWindow(Instant.ofEpochMilli(-137), Duration.millis(21L)); StateContext context = StateContexts.windowOnlyContext(window); thrown.expect(IllegalArgumentException.class); context.sideInput(view); @@ -87,7 +87,7 @@ public void windowOnlyContextThrowsOnSideInput() { @Test public void windowOnlyContextWindowReturnsWindow() { - BoundedWindow window = new IntervalWindow(new Instant(-137), Duration.millis(21L)); + BoundedWindow window = new IntervalWindow(Instant.ofEpochMilli(-137), Duration.millis(21L)); StateContext context = StateContexts.windowOnlyContext(window); assertThat(context.window(), equalTo(window)); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/GatherAllPanesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/GatherAllPanesTest.java index cae46b0d89c4..4320fa74f5d9 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/GatherAllPanesTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/GatherAllPanesTest.java @@ -54,7 +54,7 @@ public class GatherAllPanesTest implements Serializable { public void singlePaneSingleReifiedPane() { PCollection>>> accumulatedPanes = p.apply(GenerateSequence.from(0).to(20000)) - .apply(WithTimestamps.of(input -> new Instant(input * 10))) + .apply(WithTimestamps.of(input -> Instant.ofEpochMilli(input * 10))) .apply( Window.into(FixedWindows.of(Duration.standardMinutes(1))) .triggering(AfterWatermark.pastEndOfWindow()) @@ -89,7 +89,7 @@ public void multiplePanesMultipleReifiedPane() { PCollectionList.of(someElems) .and(otherElems) .apply(Flatten.pCollections()) - .apply(WithTimestamps.of(input -> new Instant(input * 10))) + .apply(WithTimestamps.of(input -> Instant.ofEpochMilli(input * 10))) .apply( Window.into(FixedWindows.of(Duration.standardMinutes(1))) .triggering( diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PAssertTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PAssertTest.java index a02196bb2c05..04666fc5c2a6 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PAssertTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PAssertTest.java @@ -222,20 +222,22 @@ public void testWindowedSerializablePredicate() throws Exception { pipeline .apply( Create.timestamped( - TimestampedValue.of(new NotSerializableObject(), new Instant(250L)), - TimestampedValue.of(new NotSerializableObject(), new Instant(500L))) + TimestampedValue.of( + new NotSerializableObject(), Instant.ofEpochMilli(250L)), + TimestampedValue.of( + new NotSerializableObject(), Instant.ofEpochMilli(500L))) .withCoder(NotSerializableObjectCoder.of())) .apply(Window.into(FixedWindows.of(Duration.millis(300L)))); PAssert.that(pcollection) - .inWindow(new IntervalWindow(new Instant(0L), new Instant(300L))) + .inWindow(new IntervalWindow(Instant.ofEpochMilli(0L), Instant.ofEpochMilli(300L))) .satisfies( contents -> { assertThat(Iterables.isEmpty(contents), is(false)); return null; // no problem! }); PAssert.that(pcollection) - .inWindow(new IntervalWindow(new Instant(300L), new Instant(600L))) + .inWindow(new IntervalWindow(Instant.ofEpochMilli(300L), Instant.ofEpochMilli(600L))) .satisfies( contents -> { assertThat(Iterables.isEmpty(contents), is(false)); @@ -335,8 +337,8 @@ public void testWindowedIsEqualTo() throws Exception { pipeline .apply( Create.timestamped( - TimestampedValue.of(43, new Instant(250L)), - TimestampedValue.of(22, new Instant(-250L)))) + TimestampedValue.of(43, Instant.ofEpochMilli(250L)), + TimestampedValue.of(22, Instant.ofEpochMilli(-250L)))) .apply(Window.into(FixedWindows.of(Duration.millis(500L)))) // Materialize final panes to be able to check for single element ON_TIME panes, // elements might be in EARLY panes otherwise. @@ -354,10 +356,10 @@ public void processElement(ProcessContext ctxt) { })); PAssert.thatSingleton(pcollection) - .inOnlyPane(new IntervalWindow(new Instant(0L), new Instant(500L))) + .inOnlyPane(new IntervalWindow(Instant.ofEpochMilli(0L), Instant.ofEpochMilli(500L))) .isEqualTo(43); PAssert.thatSingleton(pcollection) - .inOnlyPane(new IntervalWindow(new Instant(-500L), new Instant(0L))) + .inOnlyPane(new IntervalWindow(Instant.ofEpochMilli(-500L), Instant.ofEpochMilli(0L))) .isEqualTo(22); pipeline.run(); } @@ -472,10 +474,10 @@ public void testWindowedContainsInAnyOrder() throws Exception { pipeline .apply( Create.timestamped( - TimestampedValue.of(1, new Instant(100L)), - TimestampedValue.of(2, new Instant(200L)), - TimestampedValue.of(3, new Instant(300L)), - TimestampedValue.of(4, new Instant(400L)))) + TimestampedValue.of(1, Instant.ofEpochMilli(100L)), + TimestampedValue.of(2, Instant.ofEpochMilli(200L)), + TimestampedValue.of(3, Instant.ofEpochMilli(300L)), + TimestampedValue.of(4, Instant.ofEpochMilli(400L)))) .apply( Window.into( SlidingWindows.of(Duration.millis(200L)) @@ -483,19 +485,19 @@ public void testWindowedContainsInAnyOrder() throws Exception { .withOffset(Duration.millis(50L)))); PAssert.that(pcollection) - .inWindow(new IntervalWindow(new Instant(-50L), new Instant(150L))) + .inWindow(new IntervalWindow(Instant.ofEpochMilli(-50L), Instant.ofEpochMilli(150L))) .containsInAnyOrder(1); PAssert.that(pcollection) - .inWindow(new IntervalWindow(new Instant(50L), new Instant(250L))) + .inWindow(new IntervalWindow(Instant.ofEpochMilli(50L), Instant.ofEpochMilli(250L))) .containsInAnyOrder(2, 1); PAssert.that(pcollection) - .inWindow(new IntervalWindow(new Instant(150L), new Instant(350L))) + .inWindow(new IntervalWindow(Instant.ofEpochMilli(150L), Instant.ofEpochMilli(350L))) .containsInAnyOrder(2, 3); PAssert.that(pcollection) - .inWindow(new IntervalWindow(new Instant(250L), new Instant(450L))) + .inWindow(new IntervalWindow(Instant.ofEpochMilli(250L), Instant.ofEpochMilli(450L))) .containsInAnyOrder(4, 3); PAssert.that(pcollection) - .inWindow(new IntervalWindow(new Instant(350L), new Instant(550L))) + .inWindow(new IntervalWindow(Instant.ofEpochMilli(350L), Instant.ofEpochMilli(550L))) .containsInAnyOrder(4); pipeline.run(); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PaneExtractorsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PaneExtractorsTest.java index 78abbcaca872..726a9778499b 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PaneExtractorsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PaneExtractorsTest.java @@ -60,9 +60,15 @@ public void onlyPaneOnlyOneFiring() { Iterable> onlyFiring = ImmutableList.of( ValueInSingleWindow.of( - 2, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.ON_TIME_AND_ONLY_FIRING), + 2, + Instant.ofEpochMilli(0L), + GlobalWindow.INSTANCE, + PaneInfo.ON_TIME_AND_ONLY_FIRING), ValueInSingleWindow.of( - 1, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.ON_TIME_AND_ONLY_FIRING)); + 1, + Instant.ofEpochMilli(0L), + GlobalWindow.INSTANCE, + PaneInfo.ON_TIME_AND_ONLY_FIRING)); assertThat(extractor.apply(onlyFiring), containsInAnyOrder(2, 1)); } @@ -75,17 +81,17 @@ public void onlyPaneMultiplePanesFails() { ImmutableList.of( ValueInSingleWindow.of( 4, - new Instant(0L), + Instant.ofEpochMilli(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(true, false, Timing.EARLY)), ValueInSingleWindow.of( 2, - new Instant(0L), + Instant.ofEpochMilli(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(false, false, Timing.ON_TIME, 1L, 0L)), ValueInSingleWindow.of( 1, - new Instant(0L), + Instant.ofEpochMilli(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(false, false, Timing.LATE, 2L, 1L))); @@ -101,12 +107,12 @@ public void onTimePane() { ImmutableList.of( ValueInSingleWindow.of( 4, - new Instant(0L), + Instant.ofEpochMilli(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(false, false, Timing.ON_TIME, 1L, 0L)), ValueInSingleWindow.of( 2, - new Instant(0L), + Instant.ofEpochMilli(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(false, false, Timing.ON_TIME, 1L, 0L))); @@ -121,22 +127,22 @@ public void onTimePaneOnlyEarlyAndLate() { ImmutableList.of( ValueInSingleWindow.of( 8, - new Instant(0L), + Instant.ofEpochMilli(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(false, false, Timing.LATE, 2L, 1L)), ValueInSingleWindow.of( 4, - new Instant(0L), + Instant.ofEpochMilli(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(false, false, Timing.ON_TIME, 1L, 0L)), ValueInSingleWindow.of( 2, - new Instant(0L), + Instant.ofEpochMilli(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(false, false, Timing.ON_TIME, 1L, 0L)), ValueInSingleWindow.of( 1, - new Instant(0L), + Instant.ofEpochMilli(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(true, false, Timing.EARLY))); @@ -149,17 +155,17 @@ public void lateAndEarlyPaneTest() { ImmutableList.of( ValueInSingleWindow.of( 8, - new Instant(0L), + Instant.ofEpochMilli(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(false, false, Timing.LATE, 2L, 1L)), ValueInSingleWindow.of( 4, - new Instant(0L), + Instant.ofEpochMilli(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(false, false, Timing.ON_TIME, 1L, 0L)), ValueInSingleWindow.of( 1, - new Instant(0L), + Instant.ofEpochMilli(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(true, false, Timing.EARLY))); @@ -184,17 +190,17 @@ public void finalPane() { ImmutableList.of( ValueInSingleWindow.of( 8, - new Instant(0L), + Instant.ofEpochMilli(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(false, true, Timing.LATE, 2L, 1L)), ValueInSingleWindow.of( 4, - new Instant(0L), + Instant.ofEpochMilli(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(false, false, Timing.ON_TIME, 1L, 0L)), ValueInSingleWindow.of( 1, - new Instant(0L), + Instant.ofEpochMilli(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(true, false, Timing.EARLY))); @@ -209,17 +215,17 @@ public void finalPaneNoExplicitFinalEmpty() { ImmutableList.of( ValueInSingleWindow.of( 8, - new Instant(0L), + Instant.ofEpochMilli(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(false, false, Timing.LATE, 2L, 1L)), ValueInSingleWindow.of( 4, - new Instant(0L), + Instant.ofEpochMilli(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(false, false, Timing.ON_TIME, 1L, 0L)), ValueInSingleWindow.of( 1, - new Instant(0L), + Instant.ofEpochMilli(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(true, false, Timing.EARLY))); @@ -233,11 +239,20 @@ public void nonLatePanesSingleOnTime() { Iterable> onlyOnTime = ImmutableList.of( ValueInSingleWindow.of( - 8, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.ON_TIME_AND_ONLY_FIRING), + 8, + Instant.ofEpochMilli(0L), + GlobalWindow.INSTANCE, + PaneInfo.ON_TIME_AND_ONLY_FIRING), ValueInSingleWindow.of( - 4, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.ON_TIME_AND_ONLY_FIRING), + 4, + Instant.ofEpochMilli(0L), + GlobalWindow.INSTANCE, + PaneInfo.ON_TIME_AND_ONLY_FIRING), ValueInSingleWindow.of( - 2, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.ON_TIME_AND_ONLY_FIRING)); + 2, + Instant.ofEpochMilli(0L), + GlobalWindow.INSTANCE, + PaneInfo.ON_TIME_AND_ONLY_FIRING)); assertThat(extractor.apply(onlyOnTime), containsInAnyOrder(2, 4, 8)); } @@ -250,12 +265,12 @@ public void nonLatePanesSingleEarly() { ImmutableList.of( ValueInSingleWindow.of( 8, - new Instant(0L), + Instant.ofEpochMilli(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(true, false, Timing.EARLY)), ValueInSingleWindow.of( 4, - new Instant(0L), + Instant.ofEpochMilli(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(true, false, Timing.EARLY))); @@ -270,7 +285,7 @@ public void allPanesSingleLate() { ImmutableList.of( ValueInSingleWindow.of( 8, - new Instant(0L), + Instant.ofEpochMilli(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(false, false, Timing.LATE, 0L, 0L))); @@ -285,18 +300,19 @@ public void nonLatePanesMultiplePanes() { ImmutableList.of( ValueInSingleWindow.of( 8, - new Instant(0L), + Instant.ofEpochMilli(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(false, false, Timing.LATE, 2L, 1L)), - ValueInSingleWindow.of(7, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING), + ValueInSingleWindow.of( + 7, Instant.ofEpochMilli(0L), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING), ValueInSingleWindow.of( 4, - new Instant(0L), + Instant.ofEpochMilli(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(false, false, Timing.ON_TIME, 1L, 0L)), ValueInSingleWindow.of( 1, - new Instant(0L), + Instant.ofEpochMilli(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(true, false, Timing.EARLY))); @@ -310,11 +326,20 @@ public void allPanesSinglePane() { Iterable> onlyOnTime = ImmutableList.of( ValueInSingleWindow.of( - 8, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.ON_TIME_AND_ONLY_FIRING), + 8, + Instant.ofEpochMilli(0L), + GlobalWindow.INSTANCE, + PaneInfo.ON_TIME_AND_ONLY_FIRING), ValueInSingleWindow.of( - 4, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.ON_TIME_AND_ONLY_FIRING), + 4, + Instant.ofEpochMilli(0L), + GlobalWindow.INSTANCE, + PaneInfo.ON_TIME_AND_ONLY_FIRING), ValueInSingleWindow.of( - 2, new Instant(0L), GlobalWindow.INSTANCE, PaneInfo.ON_TIME_AND_ONLY_FIRING)); + 2, + Instant.ofEpochMilli(0L), + GlobalWindow.INSTANCE, + PaneInfo.ON_TIME_AND_ONLY_FIRING)); assertThat(extractor.apply(onlyOnTime), containsInAnyOrder(2, 4, 8)); } @@ -327,17 +352,17 @@ public void allPanesMultiplePanes() { ImmutableList.of( ValueInSingleWindow.of( 8, - new Instant(0L), + Instant.ofEpochMilli(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(false, false, Timing.LATE, 2L, 1L)), ValueInSingleWindow.of( 4, - new Instant(0L), + Instant.ofEpochMilli(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(false, false, Timing.ON_TIME, 1L, 0L)), ValueInSingleWindow.of( 1, - new Instant(0L), + Instant.ofEpochMilli(0L), GlobalWindow.INSTANCE, PaneInfo.createPane(true, false, Timing.EARLY))); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/StaticWindowsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/StaticWindowsTest.java index 66aa03114400..234b44758229 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/StaticWindowsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/StaticWindowsTest.java @@ -38,9 +38,10 @@ public class StaticWindowsTest { @Rule public ExpectedException thrown = ExpectedException.none(); - private final IntervalWindow first = new IntervalWindow(new Instant(0), new Instant(100_000L)); + private final IntervalWindow first = + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(100_000L)); private final IntervalWindow second = - new IntervalWindow(new Instant(1_000_000L), GlobalWindow.INSTANCE.maxTimestamp()); + new IntervalWindow(Instant.ofEpochMilli(1_000_000L), GlobalWindow.INSTANCE.maxTimestamp()); @Test public void singleWindowSucceeds() throws Exception { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java index 00b6027f945d..08339e5e467a 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java @@ -87,7 +87,7 @@ public class TestStreamTest implements Serializable { @Test @Category({ValidatesRunner.class, UsesTestStream.class}) public void testLateDataAccumulating() { - Instant instant = new Instant(0); + Instant instant = Instant.ofEpochMilli(0); TestStream source = TestStream.create(VarLongCoder.of()) .addElements( @@ -161,10 +161,10 @@ public void testProcessingTimeTrigger() { TestStream source = TestStream.create(VarLongCoder.of()) .addElements( - TimestampedValue.of(1L, new Instant(1000L)), - TimestampedValue.of(2L, new Instant(2000L))) + TimestampedValue.of(1L, Instant.ofEpochMilli(1000L)), + TimestampedValue.of(2L, Instant.ofEpochMilli(2000L))) .advanceProcessingTime(Duration.standardMinutes(12)) - .addElements(TimestampedValue.of(3L, new Instant(3000L))) + .addElements(TimestampedValue.of(3L, Instant.ofEpochMilli(3000L))) .advanceProcessingTime(Duration.standardMinutes(6)) .advanceWatermarkToInfinity(); @@ -191,15 +191,15 @@ public void testProcessingTimeTrigger() { public void testDiscardingMode() { TestStream stream = TestStream.create(StringUtf8Coder.of()) - .advanceWatermarkTo(new Instant(0)) + .advanceWatermarkTo(Instant.ofEpochMilli(0)) .addElements( - TimestampedValue.of("firstPane", new Instant(100)), - TimestampedValue.of("alsoFirstPane", new Instant(200))) - .addElements(TimestampedValue.of("onTimePane", new Instant(500))) - .advanceWatermarkTo(new Instant(1000L)) + TimestampedValue.of("firstPane", Instant.ofEpochMilli(100)), + TimestampedValue.of("alsoFirstPane", Instant.ofEpochMilli(200))) + .addElements(TimestampedValue.of("onTimePane", Instant.ofEpochMilli(500))) + .advanceWatermarkTo(Instant.ofEpochMilli(1000L)) .addElements( - TimestampedValue.of("finalLatePane", new Instant(750)), - TimestampedValue.of("alsoFinalLatePane", new Instant(250))) + TimestampedValue.of("finalLatePane", Instant.ofEpochMilli(750)), + TimestampedValue.of("alsoFinalLatePane", Instant.ofEpochMilli(250))) .advanceWatermarkToInfinity(); FixedWindows windowFn = FixedWindows.of(Duration.millis(1000L)); @@ -219,7 +219,7 @@ public void testDiscardingMode() { .apply(Values.create()) .apply(Flatten.iterables()); - IntervalWindow window = windowFn.assignWindow(new Instant(100)); + IntervalWindow window = windowFn.assignWindow(Instant.ofEpochMilli(100)); PAssert.that(values) .inWindow(window) .containsInAnyOrder( @@ -238,12 +238,12 @@ public void testDiscardingMode() { @Test @Category({ValidatesRunner.class, UsesTestStream.class}) public void testFirstElementLate() { - Instant lateElementTimestamp = new Instant(-1_000_000); + Instant lateElementTimestamp = Instant.ofEpochMilli(-1_000_000); TestStream stream = TestStream.create(StringUtf8Coder.of()) - .advanceWatermarkTo(new Instant(0)) + .advanceWatermarkTo(Instant.ofEpochMilli(0)) .addElements(TimestampedValue.of("late", lateElementTimestamp)) - .addElements(TimestampedValue.of("onTime", new Instant(100))) + .addElements(TimestampedValue.of("onTime", Instant.ofEpochMilli(100))) .advanceWatermarkToInfinity(); FixedWindows windowFn = FixedWindows.of(Duration.millis(1000L)); @@ -262,7 +262,7 @@ public void testFirstElementLate() { PAssert.that(values).inWindow(windowFn.assignWindow(lateElementTimestamp)).empty(); PAssert.that(values) - .inWindow(windowFn.assignWindow(new Instant(100))) + .inWindow(windowFn.assignWindow(Instant.ofEpochMilli(100))) .containsInAnyOrder("onTime"); p.run(); @@ -343,9 +343,9 @@ public void testElementAtPositiveInfinityThrows() { @Test public void testAdvanceWatermarkNonMonotonicThrows() { TestStream.Builder stream = - TestStream.create(VarIntCoder.of()).advanceWatermarkTo(new Instant(0L)); + TestStream.create(VarIntCoder.of()).advanceWatermarkTo(Instant.ofEpochMilli(0L)); thrown.expect(IllegalArgumentException.class); - stream.advanceWatermarkTo(new Instant(-1L)); + stream.advanceWatermarkTo(Instant.ofEpochMilli(-1L)); } @Test @@ -362,11 +362,11 @@ public void testAdvanceWatermarkEqualToPositiveInfinityThrows() { public void testEarlyPanesOfWindow() { TestStream source = TestStream.create(VarLongCoder.of()) - .addElements(TimestampedValue.of(1L, new Instant(1000L))) + .addElements(TimestampedValue.of(1L, Instant.ofEpochMilli(1000L))) .advanceProcessingTime(Duration.standardMinutes(6)) // Fire early paneInfo - .addElements(TimestampedValue.of(2L, new Instant(2000L))) + .addElements(TimestampedValue.of(2L, Instant.ofEpochMilli(2000L))) .advanceProcessingTime(Duration.standardMinutes(6)) // Fire early paneInfo - .addElements(TimestampedValue.of(3L, new Instant(3000L))) + .addElements(TimestampedValue.of(3L, Instant.ofEpochMilli(3000L))) .advanceProcessingTime(Duration.standardMinutes(6)) // Fire early paneInfo .advanceWatermarkToInfinity(); // Fire on-time paneInfo @@ -388,7 +388,8 @@ public void testEarlyPanesOfWindow() { .apply(Sum.longsPerKey()); IntervalWindow window = - new IntervalWindow(new Instant(0L), new Instant(0L).plus(Duration.standardMinutes(30))); + new IntervalWindow( + Instant.ofEpochMilli(0L), Instant.ofEpochMilli(0L).plus(Duration.standardMinutes(30))); PAssert.that(sum) .inEarlyPane(window) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/WindowSupplierTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/WindowSupplierTest.java index 8559ba540654..ecbbb4816044 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/WindowSupplierTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/WindowSupplierTest.java @@ -40,9 +40,10 @@ /** Tests for {@link WindowSupplier}. */ @RunWith(JUnit4.class) public class WindowSupplierTest { - private final IntervalWindow window = new IntervalWindow(new Instant(0L), new Instant(100L)); + private final IntervalWindow window = + new IntervalWindow(Instant.ofEpochMilli(0L), Instant.ofEpochMilli(100L)); private final IntervalWindow otherWindow = - new IntervalWindow(new Instant(-100L), new Instant(100L)); + new IntervalWindow(Instant.ofEpochMilli(-100L), Instant.ofEpochMilli(100L)); @Rule public ExpectedException thrown = ExpectedException.none(); @Test diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java index 993b84a528d7..2527183894f7 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java @@ -1065,11 +1065,11 @@ public void testFixedWindowsCombine() { pipeline .apply( Create.timestamped( - TimestampedValue.of(KV.of("a", 1), new Instant(0L)), - TimestampedValue.of(KV.of("a", 1), new Instant(1L)), - TimestampedValue.of(KV.of("a", 4), new Instant(6L)), - TimestampedValue.of(KV.of("b", 1), new Instant(7L)), - TimestampedValue.of(KV.of("b", 13), new Instant(8L))) + TimestampedValue.of(KV.of("a", 1), Instant.ofEpochMilli(0L)), + TimestampedValue.of(KV.of("a", 1), Instant.ofEpochMilli(1L)), + TimestampedValue.of(KV.of("a", 4), Instant.ofEpochMilli(6L)), + TimestampedValue.of(KV.of("b", 1), Instant.ofEpochMilli(7L)), + TimestampedValue.of(KV.of("b", 13), Instant.ofEpochMilli(8L))) .withCoder(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of()))) .apply(Window.into(FixedWindows.of(Duration.millis(2)))); @@ -1092,11 +1092,11 @@ public void testFixedWindowsCombineWithContext() { pipeline .apply( Create.timestamped( - TimestampedValue.of(KV.of("a", 1), new Instant(0L)), - TimestampedValue.of(KV.of("a", 1), new Instant(1L)), - TimestampedValue.of(KV.of("a", 4), new Instant(6L)), - TimestampedValue.of(KV.of("b", 1), new Instant(7L)), - TimestampedValue.of(KV.of("b", 13), new Instant(8L))) + TimestampedValue.of(KV.of("a", 1), Instant.ofEpochMilli(0L)), + TimestampedValue.of(KV.of("a", 1), Instant.ofEpochMilli(1L)), + TimestampedValue.of(KV.of("a", 4), Instant.ofEpochMilli(6L)), + TimestampedValue.of(KV.of("b", 1), Instant.ofEpochMilli(7L)), + TimestampedValue.of(KV.of("b", 13), Instant.ofEpochMilli(8L))) .withCoder(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of()))) .apply(Window.into(FixedWindows.of(Duration.millis(2)))); @@ -1134,9 +1134,9 @@ public void testSlidingWindowsCombine() { pipeline .apply( Create.timestamped( - TimestampedValue.of("a", new Instant(1L)), - TimestampedValue.of("b", new Instant(2L)), - TimestampedValue.of("c", new Instant(3L)))) + TimestampedValue.of("a", Instant.ofEpochMilli(1L)), + TimestampedValue.of("b", Instant.ofEpochMilli(2L)), + TimestampedValue.of("c", Instant.ofEpochMilli(3L)))) .apply(Window.into(SlidingWindows.of(Duration.millis(3)).every(Duration.millis(1L)))); PCollection> combined = input.apply( @@ -1194,11 +1194,11 @@ public void testSlidingWindowsCombineWithContext() { pipeline .apply( Create.timestamped( - TimestampedValue.of(KV.of("a", 1), new Instant(2L)), - TimestampedValue.of(KV.of("a", 1), new Instant(3L)), - TimestampedValue.of(KV.of("a", 4), new Instant(8L)), - TimestampedValue.of(KV.of("b", 1), new Instant(9L)), - TimestampedValue.of(KV.of("b", 13), new Instant(10L))) + TimestampedValue.of(KV.of("a", 1), Instant.ofEpochMilli(2L)), + TimestampedValue.of(KV.of("a", 1), Instant.ofEpochMilli(3L)), + TimestampedValue.of(KV.of("a", 4), Instant.ofEpochMilli(8L)), + TimestampedValue.of(KV.of("b", 1), Instant.ofEpochMilli(9L)), + TimestampedValue.of(KV.of("b", 13), Instant.ofEpochMilli(10L))) .withCoder(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of()))) .apply(Window.into(SlidingWindows.of(Duration.millis(2)))); @@ -1271,11 +1271,11 @@ public void testSessionsCombine() { pipeline .apply( Create.timestamped( - TimestampedValue.of(KV.of("a", 1), new Instant(0L)), - TimestampedValue.of(KV.of("a", 1), new Instant(4L)), - TimestampedValue.of(KV.of("a", 4), new Instant(7L)), - TimestampedValue.of(KV.of("b", 1), new Instant(10L)), - TimestampedValue.of(KV.of("b", 13), new Instant(16L))) + TimestampedValue.of(KV.of("a", 1), Instant.ofEpochMilli(0L)), + TimestampedValue.of(KV.of("a", 1), Instant.ofEpochMilli(4L)), + TimestampedValue.of(KV.of("a", 4), Instant.ofEpochMilli(7L)), + TimestampedValue.of(KV.of("b", 1), Instant.ofEpochMilli(10L)), + TimestampedValue.of(KV.of("b", 13), Instant.ofEpochMilli(16L))) .withCoder(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of()))) .apply(Window.into(Sessions.withGapDuration(Duration.millis(5)))); @@ -1296,11 +1296,11 @@ public void testSessionsCombineWithContext() { PCollection> perKeyInput = pipeline.apply( Create.timestamped( - TimestampedValue.of(KV.of("a", 1), new Instant(0L)), - TimestampedValue.of(KV.of("a", 1), new Instant(4L)), - TimestampedValue.of(KV.of("a", 4), new Instant(7L)), - TimestampedValue.of(KV.of("b", 1), new Instant(10L)), - TimestampedValue.of(KV.of("b", 13), new Instant(16L))) + TimestampedValue.of(KV.of("a", 1), Instant.ofEpochMilli(0L)), + TimestampedValue.of(KV.of("a", 1), Instant.ofEpochMilli(4L)), + TimestampedValue.of(KV.of("a", 4), Instant.ofEpochMilli(7L)), + TimestampedValue.of(KV.of("b", 1), Instant.ofEpochMilli(10L)), + TimestampedValue.of(KV.of("b", 13), Instant.ofEpochMilli(16L))) .withCoder(KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of()))); PCollection globallyInput = perKeyInput.apply(Values.create()); @@ -1390,12 +1390,12 @@ public void testWindowedCombineGloballyAsSingletonView() { .apply( "CreateSideInput", Create.timestamped( - TimestampedValue.of(1, new Instant(100)), - TimestampedValue.of(3, new Instant(100)))) + TimestampedValue.of(1, Instant.ofEpochMilli(100)), + TimestampedValue.of(3, Instant.ofEpochMilli(100)))) .apply("WindowSideInput", Window.into(windowFn)) .apply("CombineSideInput", Sum.integersGlobally().asSingletonView()); - TimestampedValue nonEmptyElement = TimestampedValue.of(null, new Instant(100)); + TimestampedValue nonEmptyElement = TimestampedValue.of(null, Instant.ofEpochMilli(100)); TimestampedValue emptyElement = TimestampedValue.atMinimumTimestamp(null); PCollection output = pipeline diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java index 03f4f1c67c67..11c0f9db6f21 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java @@ -177,12 +177,13 @@ public void testCreateParameterizedType() throws Exception { PCollection> output = p.apply( Create.of( - TimestampedValue.of("a", new Instant(0)), - TimestampedValue.of("b", new Instant(0)))); + TimestampedValue.of("a", Instant.ofEpochMilli(0)), + TimestampedValue.of("b", Instant.ofEpochMilli(0)))); PAssert.that(output) .containsInAnyOrder( - TimestampedValue.of("a", new Instant(0)), TimestampedValue.of("b", new Instant(0))); + TimestampedValue.of("a", Instant.ofEpochMilli(0)), + TimestampedValue.of("b", Instant.ofEpochMilli(0))); p.run(); } @@ -267,9 +268,9 @@ private static String formatMetadata(String s, Instant timestamp, BoundedWindow public void testCreateTimestamped() { List> data = Arrays.asList( - TimestampedValue.of("a", new Instant(1L)), - TimestampedValue.of("b", new Instant(2L)), - TimestampedValue.of("c", new Instant(3L))); + TimestampedValue.of("a", Instant.ofEpochMilli(1L)), + TimestampedValue.of("b", Instant.ofEpochMilli(2L)), + TimestampedValue.of("c", Instant.ofEpochMilli(3L))); PCollection output = p.apply(Create.timestamped(data)).apply(ParDo.of(new PrintTimestamps())); @@ -312,8 +313,8 @@ public void testCreateTimestampedPolymorphicType() throws Exception { PCollection c = p.apply( Create.timestamped( - TimestampedValue.of(new Record(), new Instant(0)), - TimestampedValue.of(new Record2(), new Instant(0)))); + TimestampedValue.of(new Record(), Instant.ofEpochMilli(0)), + TimestampedValue.of(new Record2(), Instant.ofEpochMilli(0)))); p.run(); @@ -325,8 +326,8 @@ public void testCreateTimestampedDefaultOutputCoderUsingCoder() throws Exception Coder coder = new RecordCoder(); Create.TimestampedValues values = Create.timestamped( - TimestampedValue.of(new Record(), new Instant(0)), - TimestampedValue.of(new Record2(), new Instant(0))) + TimestampedValue.of(new Record(), Instant.ofEpochMilli(0)), + TimestampedValue.of(new Record2(), Instant.ofEpochMilli(0))) .withCoder(coder); assertThat(p.apply(values).getCoder(), equalTo(coder)); } @@ -337,8 +338,8 @@ public void testCreateTimestampedDefaultOutputCoderUsingTypeDescriptor() throws p.getCoderRegistry().registerCoderForClass(Record.class, coder); Create.TimestampedValues values = Create.timestamped( - TimestampedValue.of(new Record(), new Instant(0)), - TimestampedValue.of(new Record2(), new Instant(0))) + TimestampedValue.of(new Record(), Instant.ofEpochMilli(0)), + TimestampedValue.of(new Record2(), Instant.ofEpochMilli(0))) .withType(new TypeDescriptor() {}); assertThat(p.apply(values).getCoder(), equalTo(coder)); } @@ -348,10 +349,15 @@ public void testCreateTimestampedDefaultOutputCoderUsingTypeDescriptor() throws public void testCreateWindowedValues() { List> data = Arrays.asList( - WindowedValues.of("a", new Instant(1L), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING), - WindowedValues.of("b", new Instant(2L), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING), WindowedValues.of( - "c", new Instant(3L), GlobalWindow.INSTANCE, PaneInfo.ON_TIME_AND_ONLY_FIRING)); + "a", Instant.ofEpochMilli(1L), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING), + WindowedValues.of( + "b", Instant.ofEpochMilli(2L), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING), + WindowedValues.of( + "c", + Instant.ofEpochMilli(3L), + GlobalWindow.INSTANCE, + PaneInfo.ON_TIME_AND_ONLY_FIRING)); // The easiest way to directly check the created PCollection with PAssert and without relying on // other diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DeduplicateTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DeduplicateTest.java index 40538ccefac0..f12e4455c797 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DeduplicateTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DeduplicateTest.java @@ -56,7 +56,7 @@ public class DeduplicateTest { @Test @Category({NeedsRunner.class, UsesTestStream.class}) public void testInDifferentWindows() { - Instant base = new Instant(0); + Instant base = Instant.ofEpochMilli(0); TestStream values = TestStream.create(StringUtf8Coder.of()) .advanceWatermarkTo(base) @@ -95,7 +95,7 @@ public void testInDifferentWindows() { @Test @Category({NeedsRunner.class, UsesTestStream.class}) public void testEventTime() { - Instant base = new Instant(0); + Instant base = Instant.ofEpochMilli(0); TestStream values = TestStream.create(StringUtf8Coder.of()) .advanceWatermarkTo(base) @@ -133,7 +133,7 @@ public void testEventTime() { @Test @Category({NeedsRunner.class, UsesTestStreamWithProcessingTime.class}) public void testProcessingTime() { - Instant base = new Instant(0); + Instant base = Instant.ofEpochMilli(0); TestStream values = TestStream.create(StringUtf8Coder.of()) .advanceWatermarkTo(base) @@ -176,7 +176,7 @@ public T apply(KV input) { @Test @Category({NeedsRunner.class, UsesTestStreamWithProcessingTime.class}) public void testRepresentativeValuesWithCoder() { - Instant base = new Instant(0); + Instant base = Instant.ofEpochMilli(0); TestStream> values = TestStream.create(KvCoder.of(VarLongCoder.of(), StringUtf8Coder.of())) .advanceWatermarkTo(base) @@ -205,7 +205,7 @@ public void testRepresentativeValuesWithCoder() { @Test @Category({NeedsRunner.class, UsesTestStreamWithProcessingTime.class}) public void testTriggeredRepresentativeValuesWithType() { - Instant base = new Instant(0); + Instant base = Instant.ofEpochMilli(0); TestStream> values = TestStream.create(KvCoder.of(VarLongCoder.of(), StringUtf8Coder.of())) .advanceWatermarkTo(base) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DistinctTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DistinctTest.java index e57074eb30dc..7b5e47749d0d 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DistinctTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DistinctTest.java @@ -138,7 +138,7 @@ public void testDistinctWithRepresentativeValue() { @Test @Category({NeedsRunner.class, UsesTestStream.class}) public void testWindowedDistinct() { - Instant base = new Instant(0); + Instant base = Instant.ofEpochMilli(0); TestStream values = TestStream.create(StringUtf8Coder.of()) .advanceWatermarkTo(base) @@ -180,7 +180,7 @@ public void testWindowedDistinct() { @Test @Category({NeedsRunner.class, UsesTestStreamWithProcessingTime.class}) public void testTriggeredDistinct() { - Instant base = new Instant(0); + Instant base = Instant.ofEpochMilli(0); TestStream values = TestStream.create(StringUtf8Coder.of()) .advanceWatermarkTo(base) @@ -216,7 +216,7 @@ public void testTriggeredDistinct() { @Test @Category({NeedsRunner.class, UsesTestStreamWithProcessingTime.class}) public void testTriggeredDistinctRepresentativeValues() { - Instant base = new Instant(0); + Instant base = Instant.ofEpochMilli(0); TestStream> values = TestStream.create(KvCoder.of(VarLongCoder.of(), StringUtf8Coder.of())) .advanceWatermarkTo(base) @@ -258,7 +258,7 @@ public void testTriggeredDistinctRepresentativeValues() { @Test @Category({NeedsRunner.class, UsesTestStreamWithProcessingTime.class}) public void testTriggeredDistinctRepresentativeValuesEmpty() { - Instant base = new Instant(0); + Instant base = Instant.ofEpochMilli(0); TestStream> values = TestStream.create(KvCoder.of(VarLongCoder.of(), StringUtf8Coder.of())) .advanceWatermarkTo(base) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java index f27b936b7110..3a637426c458 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnTesterTest.java @@ -226,7 +226,7 @@ public void clonePerBundle() throws Exception { @Test public void processTimestampedElement() throws Exception { try (DoFnTester> tester = DoFnTester.of(new ReifyTimestamps())) { - TimestampedValue input = TimestampedValue.of(1L, new Instant(100)); + TimestampedValue input = TimestampedValue.of(1L, Instant.ofEpochMilli(100)); tester.processTimestampedElement(input); assertThat(tester.takeOutputElements(), contains(input)); } @@ -246,15 +246,15 @@ public void processElementWithOutputTimestamp() throws Exception { tester.processElement(2L); List> peek = tester.peekOutputElementsWithTimestamp(); - TimestampedValue one = TimestampedValue.of("1", new Instant(1000L)); - TimestampedValue two = TimestampedValue.of("2", new Instant(2000L)); + TimestampedValue one = TimestampedValue.of("1", Instant.ofEpochMilli(1000L)); + TimestampedValue two = TimestampedValue.of("2", Instant.ofEpochMilli(2000L)); assertThat(peek, hasItems(one, two)); tester.processElement(3L); tester.processElement(4L); - TimestampedValue three = TimestampedValue.of("3", new Instant(3000L)); - TimestampedValue four = TimestampedValue.of("4", new Instant(4000L)); + TimestampedValue three = TimestampedValue.of("3", Instant.ofEpochMilli(3000L)); + TimestampedValue four = TimestampedValue.of("4", Instant.ofEpochMilli(4000L)); peek = tester.peekOutputElementsWithTimestamp(); assertThat(peek, hasItems(one, two, three, four)); List> take = tester.takeOutputElementsWithTimestamp(); @@ -282,10 +282,11 @@ public void peekValuesInWindow() throws Exception { assertThat( tester.peekOutputElementsInWindow(GlobalWindow.INSTANCE), containsInAnyOrder( - TimestampedValue.of("1", new Instant(1000L)), - TimestampedValue.of("2", new Instant(2000L)))); + TimestampedValue.of("1", Instant.ofEpochMilli(1000L)), + TimestampedValue.of("2", Instant.ofEpochMilli(2000L)))); assertThat( - tester.peekOutputElementsInWindow(new IntervalWindow(new Instant(0L), new Instant(10L))), + tester.peekOutputElementsInWindow( + new IntervalWindow(Instant.ofEpochMilli(0L), Instant.ofEpochMilli(10L))), Matchers.emptyIterable()); } } @@ -433,7 +434,7 @@ public void startBundle() { public void processElement(ProcessContext c) throws Exception { checkState(state == LifecycleState.INSIDE_BUNDLE, "Wrong state: %s", state); agg.inc(c.element()); - Instant instant = new Instant(1000L * c.element()); + Instant instant = Instant.ofEpochMilli(1000L * c.element()); c.outputWithTimestamp(c.element().toString(), instant); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java index 3ff98d47939d..d0e4699f8b0f 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupByKeyTest.java @@ -181,11 +181,11 @@ public void testCombiningAccumulatingProcessingTime() throws Exception { PCollection triggeredSums = p.apply( TestStream.create(VarIntCoder.of()) - .advanceWatermarkTo(new Instant(0)) + .advanceWatermarkTo(Instant.ofEpochMilli(0)) .addElements( - TimestampedValue.of(2, new Instant(2)), - TimestampedValue.of(5, new Instant(5))) - .advanceWatermarkTo(new Instant(100)) + TimestampedValue.of(2, Instant.ofEpochMilli(2)), + TimestampedValue.of(5, Instant.ofEpochMilli(5))) + .advanceWatermarkTo(Instant.ofEpochMilli(100)) .advanceProcessingTime(Duration.millis(10)) .advanceWatermarkToInfinity()) .apply( @@ -219,7 +219,7 @@ public void testAfterProcessingTimeContinuationTriggerEarly() throws Exception { PCollection triggeredSums = p.apply( TestStream.create(VarIntCoder.of()) - .advanceWatermarkTo(new Instant(0)) + .advanceWatermarkTo(Instant.ofEpochMilli(0)) .addElements(42) .advanceProcessingTime(Duration.millis(advanceMillis)) .advanceProcessingTime(Duration.millis(waitMillis)) @@ -505,13 +505,13 @@ public PCollection> expand(PBegin input) { public void testTimestampCombinerEarliest() { p.apply( Create.timestamped( - TimestampedValue.of(KV.of(0, "hello"), new Instant(0)), - TimestampedValue.of(KV.of(0, "goodbye"), new Instant(10)))) + TimestampedValue.of(KV.of(0, "hello"), Instant.ofEpochMilli(0)), + TimestampedValue.of(KV.of(0, "goodbye"), Instant.ofEpochMilli(10)))) .apply( Window.>into(FixedWindows.of(Duration.standardMinutes(10))) .withTimestampCombiner(TimestampCombiner.EARLIEST)) .apply(GroupByKey.create()) - .apply(ParDo.of(new AssertTimestamp(new Instant(0)))); + .apply(ParDo.of(new AssertTimestamp(Instant.ofEpochMilli(0)))); p.run(); } @@ -525,13 +525,13 @@ public void testTimestampCombinerEarliest() { public void testTimestampCombinerLatest() { p.apply( Create.timestamped( - TimestampedValue.of(KV.of(0, "hello"), new Instant(0)), - TimestampedValue.of(KV.of(0, "goodbye"), new Instant(10)))) + TimestampedValue.of(KV.of(0, "hello"), Instant.ofEpochMilli(0)), + TimestampedValue.of(KV.of(0, "goodbye"), Instant.ofEpochMilli(10)))) .apply( Window.>into(FixedWindows.of(Duration.standardMinutes(10))) .withTimestampCombiner(TimestampCombiner.LATEST)) .apply(GroupByKey.create()) - .apply(ParDo.of(new AssertTimestamp(new Instant(10)))); + .apply(ParDo.of(new AssertTimestamp(Instant.ofEpochMilli(10)))); p.run(); } @@ -761,12 +761,12 @@ public void testGroupByKeyAndWindows() { kv("k2", -33), kv("k3", 0))); PAssert.that(output) - .inWindow(new IntervalWindow(new Instant(0L), Duration.millis(5L))) + .inWindow(new IntervalWindow(Instant.ofEpochMilli(0L), Duration.millis(5L))) .satisfies( containsKvs( kv("k1", 3), kv("k5", Integer.MIN_VALUE, Integer.MAX_VALUE), kv("k2", 66))); PAssert.that(output) - .inWindow(new IntervalWindow(new Instant(5L), Duration.millis(5L))) + .inWindow(new IntervalWindow(Instant.ofEpochMilli(5L), Duration.millis(5L))) .satisfies(containsKvs(kv("k1", 4), kv("k2", -33), kv("k3", 0))); p.run(); @@ -778,9 +778,9 @@ public void testGroupByKeyMultipleWindows() { PCollection> windowedInput = p.apply( Create.timestamped( - TimestampedValue.of(KV.of("foo", 1), new Instant(1)), - TimestampedValue.of(KV.of("foo", 4), new Instant(4)), - TimestampedValue.of(KV.of("bar", 3), new Instant(3)))) + TimestampedValue.of(KV.of("foo", 1), Instant.ofEpochMilli(1)), + TimestampedValue.of(KV.of("foo", 4), Instant.ofEpochMilli(4)), + TimestampedValue.of(KV.of("bar", 3), Instant.ofEpochMilli(3)))) .apply( Window.into(SlidingWindows.of(Duration.millis(5L)).every(Duration.millis(3L)))); @@ -790,13 +790,13 @@ public void testGroupByKeyMultipleWindows() { .satisfies( containsKvs(kv("foo", 1, 4), kv("foo", 1), kv("foo", 4), kv("bar", 3), kv("bar", 3))); PAssert.that(output) - .inWindow(new IntervalWindow(new Instant(-3L), Duration.millis(5L))) + .inWindow(new IntervalWindow(Instant.ofEpochMilli(-3L), Duration.millis(5L))) .satisfies(containsKvs(kv("foo", 1))); PAssert.that(output) - .inWindow(new IntervalWindow(new Instant(0L), Duration.millis(5L))) + .inWindow(new IntervalWindow(Instant.ofEpochMilli(0L), Duration.millis(5L))) .satisfies(containsKvs(kv("foo", 1, 4), kv("bar", 3))); PAssert.that(output) - .inWindow(new IntervalWindow(new Instant(3L), Duration.millis(5L))) + .inWindow(new IntervalWindow(Instant.ofEpochMilli(3L), Duration.millis(5L))) .satisfies(containsKvs(kv("foo", 4), kv("bar", 3))); p.run(); @@ -808,23 +808,23 @@ public void testGroupByKeyMergingWindows() { PCollection> windowedInput = p.apply( Create.timestamped( - TimestampedValue.of(KV.of("foo", 1), new Instant(1)), - TimestampedValue.of(KV.of("foo", 4), new Instant(4)), - TimestampedValue.of(KV.of("bar", 3), new Instant(3)), - TimestampedValue.of(KV.of("foo", 9), new Instant(9)))) + TimestampedValue.of(KV.of("foo", 1), Instant.ofEpochMilli(1)), + TimestampedValue.of(KV.of("foo", 4), Instant.ofEpochMilli(4)), + TimestampedValue.of(KV.of("bar", 3), Instant.ofEpochMilli(3)), + TimestampedValue.of(KV.of("foo", 9), Instant.ofEpochMilli(9)))) .apply(Window.into(Sessions.withGapDuration(Duration.millis(4L)))); PCollection>> output = windowedInput.apply(GroupByKey.create()); PAssert.that(output).satisfies(containsKvs(kv("foo", 1, 4), kv("foo", 9), kv("bar", 3))); PAssert.that(output) - .inWindow(new IntervalWindow(new Instant(1L), new Instant(8L))) + .inWindow(new IntervalWindow(Instant.ofEpochMilli(1L), Instant.ofEpochMilli(8L))) .satisfies(containsKvs(kv("foo", 1, 4))); PAssert.that(output) - .inWindow(new IntervalWindow(new Instant(3L), new Instant(7L))) + .inWindow(new IntervalWindow(Instant.ofEpochMilli(3L), Instant.ofEpochMilli(7L))) .satisfies(containsKvs(kv("bar", 3))); PAssert.that(output) - .inWindow(new IntervalWindow(new Instant(9L), new Instant(13L))) + .inWindow(new IntervalWindow(Instant.ofEpochMilli(9L), Instant.ofEpochMilli(13L))) .satisfies(containsKvs(kv("foo", 9))); p.run(); @@ -836,10 +836,10 @@ public void testRewindowWithTimestampCombiner() { PCollection> input = p.apply( Create.timestamped( - TimestampedValue.of(KV.of("foo", 1), new Instant(1)), - TimestampedValue.of(KV.of("foo", 4), new Instant(4)), - TimestampedValue.of(KV.of("bar", 3), new Instant(3)), - TimestampedValue.of(KV.of("foo", 9), new Instant(9)))) + TimestampedValue.of(KV.of("foo", 1), Instant.ofEpochMilli(1)), + TimestampedValue.of(KV.of("foo", 4), Instant.ofEpochMilli(4)), + TimestampedValue.of(KV.of("bar", 3), Instant.ofEpochMilli(3)), + TimestampedValue.of(KV.of("foo", 9), Instant.ofEpochMilli(9)))) .apply( "GlobalWindows", Window.>configure() @@ -856,9 +856,9 @@ public void testRewindowWithTimestampCombiner() { .apply("FixedWindows", Window.into(FixedWindows.of(Duration.millis(1)))); PAssert.that(result) - .inWindow(new IntervalWindow(new Instant(9), new Instant(10))) + .inWindow(new IntervalWindow(Instant.ofEpochMilli(9), Instant.ofEpochMilli(10))) .containsInAnyOrder(KV.of("foo", 14)) - .inWindow(new IntervalWindow(new Instant(3), new Instant(4))) + .inWindow(new IntervalWindow(Instant.ofEpochMilli(3), Instant.ofEpochMilli(4))) .containsInAnyOrder(KV.of("bar", 3)); p.run(); @@ -897,10 +897,10 @@ public void testWindowFnPostMerging() throws Exception { PCollection> windowedInput = p.apply( Create.timestamped( - TimestampedValue.of(KV.of("foo", 1), new Instant(1)), - TimestampedValue.of(KV.of("foo", 4), new Instant(4)), - TimestampedValue.of(KV.of("bar", 3), new Instant(3)), - TimestampedValue.of(KV.of("foo", 9), new Instant(9)))) + TimestampedValue.of(KV.of("foo", 1), Instant.ofEpochMilli(1)), + TimestampedValue.of(KV.of("foo", 4), Instant.ofEpochMilli(4)), + TimestampedValue.of(KV.of("bar", 3), Instant.ofEpochMilli(3)), + TimestampedValue.of(KV.of("foo", 9), Instant.ofEpochMilli(9)))) .apply(Window.into(Sessions.withGapDuration(Duration.millis(4L)))); PCollection>> grouped = diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupIntoBatchesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupIntoBatchesTest.java index 832eb03f05d1..4611e550f8aa 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupIntoBatchesTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/GroupIntoBatchesTest.java @@ -343,7 +343,7 @@ public Void apply(Iterable>> input) { }) public void testInStreamingMode() { int timestampInterval = 1; - Instant startInstant = new Instant(0L); + Instant startInstant = Instant.ofEpochMilli(0L); TestStream.Builder> streamBuilder = TestStream.create(KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of())) .advanceWatermarkTo(startInstant); @@ -452,7 +452,7 @@ public void testBufferingTimerInFixedWindow() { final Duration windowDuration = Duration.standardSeconds(4); final Duration maxBufferingDuration = Duration.standardSeconds(5); - Instant startInstant = new Instant(0L); + Instant startInstant = Instant.ofEpochMilli(0L); TestStream.Builder> streamBuilder = TestStream.create(KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of())) .advanceWatermarkTo(startInstant); @@ -576,7 +576,7 @@ public void processElement(ProcessContext c, BoundedWindow window) { public void testBufferingTimerInGlobalWindow() { final Duration maxBufferingDuration = Duration.standardSeconds(5); - Instant startInstant = new Instant(0L); + Instant startInstant = Instant.ofEpochMilli(0L); long offset = 0L; int timestampInterval = 1; List>> events = new ArrayList<>(); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestFnTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestFnTest.java index 3eafbaa7c862..846193ed38f0 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestFnTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestFnTest.java @@ -41,7 +41,7 @@ /** Unit tests for {@link Latest.LatestFn}. */ @RunWith(JUnit4.class) public class LatestFnTest { - private static final Instant INSTANT = new Instant(100); + private static final Instant INSTANT = Instant.ofEpochMilli(100); private static final long VALUE = 100 * INSTANT.getMillis(); private static final TimestampedValue TV = TimestampedValue.of(VALUE, INSTANT); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestTest.java index 20b04d88bdab..9de0dffc7d5b 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestTest.java @@ -55,9 +55,9 @@ public void testGloballyEventTimestamp() { PCollection output = p.apply( Create.timestamped( - TimestampedValue.of("foo", new Instant(100)), - TimestampedValue.of("bar", new Instant(300)), - TimestampedValue.of("baz", new Instant(200)))) + TimestampedValue.of("foo", Instant.ofEpochMilli(100)), + TimestampedValue.of("bar", Instant.ofEpochMilli(300)), + TimestampedValue.of("baz", Instant.ofEpochMilli(200)))) .apply(Latest.globally()); PAssert.that(output).containsInAnyOrder("bar"); @@ -94,9 +94,9 @@ public void testPerKeyEventTimestamp() { PCollection> output = p.apply( Create.timestamped( - TimestampedValue.of(KV.of("A", "foo"), new Instant(100)), - TimestampedValue.of(KV.of("B", "bar"), new Instant(300)), - TimestampedValue.of(KV.of("A", "baz"), new Instant(200)))) + TimestampedValue.of(KV.of("A", "foo"), Instant.ofEpochMilli(100)), + TimestampedValue.of(KV.of("B", "bar"), Instant.ofEpochMilli(300)), + TimestampedValue.of(KV.of("A", "baz"), Instant.ofEpochMilli(200)))) .apply(Latest.perKey()); PAssert.that(output).containsInAnyOrder(KV.of("B", "bar"), KV.of("A", "baz")); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MapViewTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MapViewTest.java index 005feda63ab9..5d77d41b6314 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MapViewTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MapViewTest.java @@ -267,10 +267,10 @@ public void testWindowedMultimapSideInput() { .apply( "CreateSideInput", Create.timestamped( - TimestampedValue.of(KV.of("a", 1), new Instant(1)), - TimestampedValue.of(KV.of("a", 1), new Instant(2)), - TimestampedValue.of(KV.of("a", 2), new Instant(7)), - TimestampedValue.of(KV.of("b", 3), new Instant(14)))) + TimestampedValue.of(KV.of("a", 1), Instant.ofEpochMilli(1)), + TimestampedValue.of(KV.of("a", 1), Instant.ofEpochMilli(2)), + TimestampedValue.of(KV.of("a", 2), Instant.ofEpochMilli(7)), + TimestampedValue.of(KV.of("b", 3), Instant.ofEpochMilli(14)))) .apply("SideWindowInto", Window.into(FixedWindows.of(Duration.millis(10)))) .apply(View.asMultimap()); @@ -279,9 +279,9 @@ public void testWindowedMultimapSideInput() { .apply( "CreateMainInput", Create.timestamped( - TimestampedValue.of("apple", new Instant(5)), - TimestampedValue.of("banana", new Instant(13)), - TimestampedValue.of("blackberry", new Instant(16)))) + TimestampedValue.of("apple", Instant.ofEpochMilli(5)), + TimestampedValue.of("banana", Instant.ofEpochMilli(13)), + TimestampedValue.of("blackberry", Instant.ofEpochMilli(16)))) .apply("MainWindowInto", Window.into(FixedWindows.of(Duration.millis(10)))) .apply( "OutputSideInputs", @@ -316,10 +316,10 @@ public void testWindowedMultimapAsEntrySetSideInput() { .apply( "CreateSideInput", Create.timestamped( - TimestampedValue.of(KV.of("a", 1), new Instant(1)), - TimestampedValue.of(KV.of("a", 1), new Instant(2)), - TimestampedValue.of(KV.of("a", 2), new Instant(7)), - TimestampedValue.of(KV.of("b", 3), new Instant(14)))) + TimestampedValue.of(KV.of("a", 1), Instant.ofEpochMilli(1)), + TimestampedValue.of(KV.of("a", 1), Instant.ofEpochMilli(2)), + TimestampedValue.of(KV.of("a", 2), Instant.ofEpochMilli(7)), + TimestampedValue.of(KV.of("b", 3), Instant.ofEpochMilli(14)))) .apply("SideWindowInto", Window.into(FixedWindows.of(Duration.millis(10)))) .apply(View.asMultimap()); @@ -328,8 +328,8 @@ public void testWindowedMultimapAsEntrySetSideInput() { .apply( "CreateMainInput", Create.timestamped( - TimestampedValue.of(1 /* size */, new Instant(5)), - TimestampedValue.of(1 /* size */, new Instant(16)))) + TimestampedValue.of(1 /* size */, Instant.ofEpochMilli(5)), + TimestampedValue.of(1 /* size */, Instant.ofEpochMilli(16)))) .apply("MainWindowInto", Window.into(FixedWindows.of(Duration.millis(10)))) .apply( "OutputSideInputs", @@ -364,10 +364,10 @@ public void testWindowedMultimapSideInputWithNonDeterministicKeyCoder() { .apply( "CreateSideInput", Create.timestamped( - TimestampedValue.of(KV.of("a", 1), new Instant(1)), - TimestampedValue.of(KV.of("a", 1), new Instant(2)), - TimestampedValue.of(KV.of("a", 2), new Instant(7)), - TimestampedValue.of(KV.of("b", 3), new Instant(14))) + TimestampedValue.of(KV.of("a", 1), Instant.ofEpochMilli(1)), + TimestampedValue.of(KV.of("a", 1), Instant.ofEpochMilli(2)), + TimestampedValue.of(KV.of("a", 2), Instant.ofEpochMilli(7)), + TimestampedValue.of(KV.of("b", 3), Instant.ofEpochMilli(14))) .withCoder(KvCoder.of(new NonDeterministicStringCoder(), VarIntCoder.of()))) .apply("SideWindowInto", Window.into(FixedWindows.of(Duration.millis(10)))) .apply(View.asMultimap()); @@ -377,9 +377,9 @@ public void testWindowedMultimapSideInputWithNonDeterministicKeyCoder() { .apply( "CreateMainInput", Create.timestamped( - TimestampedValue.of("apple", new Instant(5)), - TimestampedValue.of("banana", new Instant(13)), - TimestampedValue.of("blackberry", new Instant(16)))) + TimestampedValue.of("apple", Instant.ofEpochMilli(5)), + TimestampedValue.of("banana", Instant.ofEpochMilli(13)), + TimestampedValue.of("blackberry", Instant.ofEpochMilli(16)))) .apply("MainWindowInto", Window.into(FixedWindows.of(Duration.millis(10)))) .apply( "OutputSideInputs", @@ -701,9 +701,9 @@ public void testWindowedMapSideInput() { .apply( "CreateSideInput", Create.timestamped( - TimestampedValue.of(KV.of("a", 1), new Instant(1)), - TimestampedValue.of(KV.of("b", 2), new Instant(4)), - TimestampedValue.of(KV.of("b", 3), new Instant(18)))) + TimestampedValue.of(KV.of("a", 1), Instant.ofEpochMilli(1)), + TimestampedValue.of(KV.of("b", 2), Instant.ofEpochMilli(4)), + TimestampedValue.of(KV.of("b", 3), Instant.ofEpochMilli(18)))) .apply("SideWindowInto", Window.into(FixedWindows.of(Duration.millis(10)))) .apply(View.asMap()); @@ -712,9 +712,9 @@ public void testWindowedMapSideInput() { .apply( "CreateMainInput", Create.timestamped( - TimestampedValue.of("apple", new Instant(5)), - TimestampedValue.of("banana", new Instant(4)), - TimestampedValue.of("blackberry", new Instant(16)))) + TimestampedValue.of("apple", Instant.ofEpochMilli(5)), + TimestampedValue.of("banana", Instant.ofEpochMilli(4)), + TimestampedValue.of("blackberry", Instant.ofEpochMilli(16)))) .apply("MainWindowInto", Window.into(FixedWindows.of(Duration.millis(10)))) .apply( "OutputSideInputs", @@ -745,9 +745,9 @@ public void testWindowedMapAsEntrySetSideInput() { .apply( "CreateSideInput", Create.timestamped( - TimestampedValue.of(KV.of("a", 1), new Instant(1)), - TimestampedValue.of(KV.of("b", 2), new Instant(4)), - TimestampedValue.of(KV.of("b", 3), new Instant(18)))) + TimestampedValue.of(KV.of("a", 1), Instant.ofEpochMilli(1)), + TimestampedValue.of(KV.of("b", 2), Instant.ofEpochMilli(4)), + TimestampedValue.of(KV.of("b", 3), Instant.ofEpochMilli(18)))) .apply("SideWindowInto", Window.into(FixedWindows.of(Duration.millis(10)))) .apply(View.asMap()); @@ -756,8 +756,8 @@ public void testWindowedMapAsEntrySetSideInput() { .apply( "CreateMainInput", Create.timestamped( - TimestampedValue.of(2 /* size */, new Instant(5)), - TimestampedValue.of(1 /* size */, new Instant(16)))) + TimestampedValue.of(2 /* size */, Instant.ofEpochMilli(5)), + TimestampedValue.of(1 /* size */, Instant.ofEpochMilli(16)))) .apply("MainWindowInto", Window.into(FixedWindows.of(Duration.millis(10)))) .apply( "OutputSideInputs", @@ -788,9 +788,9 @@ public void testWindowedMapSideInputWithNonDeterministicKeyCoder() { .apply( "CreateSideInput", Create.timestamped( - TimestampedValue.of(KV.of("a", 1), new Instant(1)), - TimestampedValue.of(KV.of("b", 2), new Instant(4)), - TimestampedValue.of(KV.of("b", 3), new Instant(18))) + TimestampedValue.of(KV.of("a", 1), Instant.ofEpochMilli(1)), + TimestampedValue.of(KV.of("b", 2), Instant.ofEpochMilli(4)), + TimestampedValue.of(KV.of("b", 3), Instant.ofEpochMilli(18))) .withCoder(KvCoder.of(new NonDeterministicStringCoder(), VarIntCoder.of()))) .apply("SideWindowInto", Window.into(FixedWindows.of(Duration.millis(10)))) .apply(View.asMap()); @@ -800,9 +800,9 @@ public void testWindowedMapSideInputWithNonDeterministicKeyCoder() { .apply( "CreateMainInput", Create.timestamped( - TimestampedValue.of("apple", new Instant(5)), - TimestampedValue.of("banana", new Instant(4)), - TimestampedValue.of("blackberry", new Instant(16)))) + TimestampedValue.of("apple", Instant.ofEpochMilli(5)), + TimestampedValue.of("banana", Instant.ofEpochMilli(4)), + TimestampedValue.of("blackberry", Instant.ofEpochMilli(16)))) .apply("MainWindowInto", Window.into(FixedWindows.of(Duration.millis(10)))) .apply( "OutputSideInputs", diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java index 8a273127b4fc..a587a2435b20 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java @@ -325,7 +325,7 @@ public void processElement(ProcessContext c) {} static class TestOutputTimestampDoFn extends DoFn { @ProcessElement public void processElement(@Element T value, OutputReceiver r) { - r.outputWithTimestamp(value, new Instant(value.longValue())); + r.outputWithTimestamp(value, Instant.ofEpochMilli(value.longValue())); } } @@ -1644,7 +1644,7 @@ public void testBundleFinalization() { // Advance the time, and add the final element. This allows Finalization // check mechanism to work without being sensitive to how bundles are // produced by a runner. - stream = stream.advanceWatermarkTo(new Instant(10)); + stream = stream.advanceWatermarkTo(Instant.ofEpochMilli(10)); stream = stream.addElements(KV.of("key" + (attemptCap % 10), attemptCap)); PCollection output = pipeline @@ -1792,7 +1792,7 @@ public void testWindowingInStartAndFinishBundle() { final FixedWindows windowFn = FixedWindows.of(Duration.millis(1)); PCollection output = pipeline - .apply(Create.timestamped(TimestampedValue.of("elem", new Instant(1)))) + .apply(Create.timestamped(TimestampedValue.of("elem", Instant.ofEpochMilli(1)))) .apply(Window.into(windowFn)) .apply( ParDo.of( @@ -1807,7 +1807,7 @@ public void processElement( @FinishBundle public void finishBundle(FinishBundleContext c) { - Instant ts = new Instant(3); + Instant ts = Instant.ofEpochMilli(3); c.output("finish", ts, windowFn.assignWindow(ts)); } })) @@ -1876,7 +1876,7 @@ public void processElement(ProcessContext context, @TimerId(TIMER_ID) Timer time } } try { - timer.withOutputTimestamp(outputTimestamp).set(new Instant(0)); + timer.withOutputTimestamp(outputTimestamp).set(Instant.ofEpochMilli(0)); context.output(TIMER_ELEMENT); } catch (IllegalArgumentException e) { if (hasExpectedError(e, allowedSkew, context.timestamp(), outputTimestamp)) { @@ -2031,7 +2031,8 @@ public void testParDoTaggedOutputWithTimestamp() { public void processElement( @Element Integer element, MultiOutputReceiver r) { r.get(additionalOutputTag) - .outputWithTimestamp(element, new Instant(element.longValue())); + .outputWithTimestamp( + element, Instant.ofEpochMilli(element.longValue())); } }) .withOutputTags(mainOutputTag, TupleTagList.of(additionalOutputTag))) @@ -2171,7 +2172,8 @@ public void testProcessElementSkew() { .apply("createNoSkew", input) .apply("noSkew", ParDo.of(new ProcessElementTimestampSkewingDoFn(Duration.ZERO))); PAssert.that(noSkew) - .containsInAnyOrder(TIMER_ELEMENT + new Instant(0L), OUTPUT_ELEMENT + new Instant(0L)); + .containsInAnyOrder( + TIMER_ELEMENT + Instant.ofEpochMilli(0L), OUTPUT_ELEMENT + Instant.ofEpochMilli(0L)); PCollection skew = pipeline @@ -2192,8 +2194,8 @@ public void testProcessElementSkew() { .apply("largeSkew", ParDo.of(new ProcessElementTimestampSkewingDoFn(allowedSkew))); PAssert.that(largeSkew) .containsInAnyOrder( - TIMER_ELEMENT + new Instant(0L).minus(offset), - OUTPUT_ELEMENT + new Instant(0L).minus(offset)); + TIMER_ELEMENT + Instant.ofEpochMilli(0L).minus(offset), + OUTPUT_ELEMENT + Instant.ofEpochMilli(0L).minus(offset)); pipeline.run(); } @@ -2211,7 +2213,9 @@ public void testOnTimerTimestampSkew() { ParDo.of( new OnTimerTimestampSkewingDoFn(Duration.millis(0L), Duration.millis(3L)))); PAssert.that(noSkew) - .containsInAnyOrder(OUTPUT_ELEMENT + new Instant(-3L), TIMER_ELEMENT + new Instant(-3L)); + .containsInAnyOrder( + OUTPUT_ELEMENT + Instant.ofEpochMilli(-3L), + TIMER_ELEMENT + Instant.ofEpochMilli(-3L)); PCollection skew = pipeline .apply("createSkew", input) @@ -2231,8 +2235,8 @@ public void testOnTimerTimestampSkew() { .apply("largeSkew", ParDo.of(new OnTimerTimestampSkewingDoFn(allowedSkew, offset))); PAssert.that(largeSkew) .containsInAnyOrder( - TIMER_ELEMENT + new Instant(0L).minus(offset), - OUTPUT_ELEMENT + new Instant(0L).minus(offset)); + TIMER_ELEMENT + Instant.ofEpochMilli(0L).minus(offset), + OUTPUT_ELEMENT + Instant.ofEpochMilli(0L).minus(offset)); pipeline.run(); } @@ -2256,7 +2260,7 @@ public void testOnWindowTimestampSkew() { PAssert.that(noSkew) .containsInAnyOrder( OUTPUT_ELEMENT - + new Instant( + + Instant.ofEpochMilli( windowDuration .minus(Duration.millis(3L)) .minus(Duration.millis(1L)) @@ -2468,21 +2472,23 @@ public void processElement( } }; - IntervalWindow firstWindow = new IntervalWindow(new Instant(0), new Instant(10)); - IntervalWindow secondWindow = new IntervalWindow(new Instant(10), new Instant(20)); + IntervalWindow firstWindow = + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10)); + IntervalWindow secondWindow = + new IntervalWindow(Instant.ofEpochMilli(10), Instant.ofEpochMilli(20)); PCollection output = pipeline .apply( Create.timestamped( // first window - TimestampedValue.of(KV.of("hello", 7), new Instant(1)), - TimestampedValue.of(KV.of("hello", 14), new Instant(2)), - TimestampedValue.of(KV.of("hello", 21), new Instant(3)), + TimestampedValue.of(KV.of("hello", 7), Instant.ofEpochMilli(1)), + TimestampedValue.of(KV.of("hello", 14), Instant.ofEpochMilli(2)), + TimestampedValue.of(KV.of("hello", 21), Instant.ofEpochMilli(3)), // second window - TimestampedValue.of(KV.of("hello", 28), new Instant(11)), - TimestampedValue.of(KV.of("hello", 35), new Instant(13)))) + TimestampedValue.of(KV.of("hello", 28), Instant.ofEpochMilli(11)), + TimestampedValue.of(KV.of("hello", 35), Instant.ofEpochMilli(13)))) .apply(Window.into(FixedWindows.of(Duration.millis(10)))) .apply("Stateful ParDo", ParDo.of(fn)); @@ -2813,7 +2819,7 @@ public void processElement( TestStream.create( KvCoder.of( StringUtf8Coder.of(), KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()))) - .advanceWatermarkTo(new Instant(0)) + .advanceWatermarkTo(Instant.ofEpochMilli(0)) .addElements(KV.of("hello", KV.of("a", 97)), KV.of("hello", KV.of("b", 42))) .addElements(KV.of("hello", KV.of("b", 33)), KV.of("hello", KV.of("a", 12))) .advanceWatermarkToInfinity(); @@ -3329,7 +3335,7 @@ public void processElement( }; TestStream> stream = TestStream.create(KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of())) - .advanceWatermarkTo(new Instant(0)) + .advanceWatermarkTo(Instant.ofEpochMilli(0)) .addElements(KV.of("hello", 1), KV.of("hello", 2)) .addElements(KV.of("hello", 2), KV.of("hello", 3)) .advanceWatermarkToInfinity(); @@ -4694,17 +4700,21 @@ public TypeDescriptor getOutputTypeDescriptor() { SlidingWindows.of(Duration.standardMinutes(3)).every(Duration.standardMinutes(1)); PCollection output = pipeline - .apply(Create.timestamped(TimestampedValue.of(KV.of("hello", 24), new Instant(0L)))) + .apply( + Create.timestamped( + TimestampedValue.of(KV.of("hello", 24), Instant.ofEpochMilli(0L)))) .apply(Window.into(windowing)) .apply(ParDo.of(fn)); PAssert.that(output) .containsInAnyOrder( - new IntervalWindow(new Instant(0), Duration.standardMinutes(3)), + new IntervalWindow(Instant.ofEpochMilli(0), Duration.standardMinutes(3)), new IntervalWindow( - new Instant(0).minus(Duration.standardMinutes(1)), Duration.standardMinutes(3)), + Instant.ofEpochMilli(0).minus(Duration.standardMinutes(1)), + Duration.standardMinutes(3)), new IntervalWindow( - new Instant(0).minus(Duration.standardMinutes(2)), Duration.standardMinutes(3))); + Instant.ofEpochMilli(0).minus(Duration.standardMinutes(2)), + Duration.standardMinutes(3))); pipeline.run(); } @@ -4953,11 +4963,13 @@ public void onTimer(@Timestamp Instant timestamp, OutputReceiver o) { PCollection output = pipeline - .apply(Create.timestamped(TimestampedValue.of(KV.of("hello", 1L), new Instant(3)))) + .apply( + Create.timestamped( + TimestampedValue.of(KV.of("hello", 1L), Instant.ofEpochMilli(3)))) .setIsBoundedInternal(useStreaming ? IsBounded.UNBOUNDED : IsBounded.BOUNDED) .apply("first", ParDo.of(fn1)); - PAssert.that(output).containsInAnyOrder(new Instant(8).getMillis()); // result output + PAssert.that(output).containsInAnyOrder(Instant.ofEpochMilli(8).getMillis()); // result output pipeline.run(); } @@ -5005,7 +5017,9 @@ public void onTimer(@Timestamp Instant timestamp, OutputReceiver o) { } PCollection output = pipeline - .apply(Create.timestamped(TimestampedValue.of(KV.of("hello", 1L), new Instant(3)))) + .apply( + Create.timestamped( + TimestampedValue.of(KV.of("hello", 1L), Instant.ofEpochMilli(3)))) .apply("first", ParDo.of(fn1)); pipeline.run(); @@ -5166,9 +5180,9 @@ public void onTimer(OutputReceiver r) { TestStream> stream = TestStream.create(KvCoder.of(StringUtf8Coder.of(), VarLongCoder.of())) - .advanceWatermarkTo(new Instant(0)) + .advanceWatermarkTo(Instant.ofEpochMilli(0)) .addElements(KV.of("hello", 37L)) - .advanceWatermarkTo(new Instant(0).plus(Duration.standardSeconds(1))) + .advanceWatermarkTo(Instant.ofEpochMilli(0).plus(Duration.standardSeconds(1))) .advanceWatermarkToInfinity(); PCollection output = pipeline.apply(stream).apply(ParDo.of(fn)); @@ -5207,18 +5221,18 @@ public void onTimer(@Timestamp Instant timestamp, OutputReceiver> stream = TestStream.create(KvCoder.of(StringUtf8Coder.of(), VarLongCoder.of())) - .advanceWatermarkTo(new Instant(0).plus(Duration.standardSeconds(5))) + .advanceWatermarkTo(Instant.ofEpochMilli(0).plus(Duration.standardSeconds(5))) .addElements(KV.of("hello", 37L)) - .advanceWatermarkTo(new Instant(0).plus(Duration.standardMinutes(1))) + .advanceWatermarkTo(Instant.ofEpochMilli(0).plus(Duration.standardMinutes(1))) .advanceWatermarkToInfinity(); PCollection> output = pipeline.apply(stream).apply(ParDo.of(fn)); PAssert.that(output) .containsInAnyOrder( - KV.of(3L, new Instant(0).plus(Duration.standardSeconds(5))), + KV.of(3L, Instant.ofEpochMilli(0).plus(Duration.standardSeconds(5))), KV.of( 42L, - new Instant( + Instant.ofEpochMilli( Duration.standardMinutes(1).minus(Duration.standardSeconds(1)).getMillis()))); pipeline.run(); } @@ -5251,9 +5265,9 @@ public void onTimer(@Timestamp Instant timestamp, OutputReceiver> stream = TestStream.create(KvCoder.of(StringUtf8Coder.of(), VarLongCoder.of())) - .advanceWatermarkTo(new Instant(0).plus(Duration.standardSeconds(5))) + .advanceWatermarkTo(Instant.ofEpochMilli(0).plus(Duration.standardSeconds(5))) .addElements(KV.of("hello", 37L)) - .advanceWatermarkTo(new Instant(0).plus(Duration.standardMinutes(1))) + .advanceWatermarkTo(Instant.ofEpochMilli(0).plus(Duration.standardMinutes(1))) .advanceWatermarkToInfinity(); PCollection> output = @@ -5266,8 +5280,8 @@ public void onTimer(@Timestamp Instant timestamp, OutputReceiver r) { TestStream> stream = TestStream.create(KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of())) - .advanceWatermarkTo(new Instant(0)) + .advanceWatermarkTo(Instant.ofEpochMilli(0)) .addElements(KV.of("hello", "input1")) .addElements(KV.of("hello", "input2")) .advanceWatermarkToInfinity(); @@ -5413,10 +5427,10 @@ public void onTimer(OutputReceiver r) { }) public void testEventTimeTimerOrdering() throws Exception { final int numTestElements = 10; - final Instant now = new Instant(0); + final Instant now = Instant.ofEpochMilli(0); TestStream.Builder> builder = TestStream.create(KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of())) - .advanceWatermarkTo(new Instant(0)); + .advanceWatermarkTo(Instant.ofEpochMilli(0)); for (int i = 0; i < numTestElements; i++) { builder = @@ -5441,7 +5455,7 @@ public void testEventTimeTimerOrdering() throws Exception { }) public void testEventTimeTimerOrderingWithCreate() throws Exception { final int numTestElements = 100; - final Instant now = new Instant(0L); + final Instant now = Instant.ofEpochMilli(0L); List>> elements = new ArrayList<>(); for (int i = 0; i < numTestElements; i++) { @@ -5627,7 +5641,7 @@ public void testTwoTimersSettingEachOtherUnbounded() { } private void testTwoTimersSettingEachOther(IsBounded isBounded) { - Instant now = new Instant(1500000000000L); + Instant now = Instant.ofEpochMilli(1500000000000L); Instant end = now.plus(Duration.millis(100)); TestStream> input = TestStream.create(KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of())) @@ -5655,7 +5669,7 @@ public void testTwoTimersSettingEachOtherWithCreateAsInputUnbounded() { } private void testTwoTimersSettingEachOtherWithCreateAsInput(IsBounded isBounded) { - Instant now = new Instant(0L); + Instant now = Instant.ofEpochMilli(0L); Instant end = now.plus(Duration.millis(100)); pipeline.apply(TwoTimerTest.of(now, end, Create.of(KV.of("", "")), isBounded)); pipeline.run(); @@ -5727,12 +5741,12 @@ public void onTimer( TestStream> stream = TestStream.create(KvCoder.of(StringUtf8Coder.of(), VarLongCoder.of())) - .advanceWatermarkTo(new Instant(0)) + .advanceWatermarkTo(Instant.ofEpochMilli(0)) // Cause fn2 to set a timer. .addElements(KV.of("key", 1L)) // Normally this would case fn2's timer to expire, but it shouldn't here because of // the output timestamp. - .advanceWatermarkTo(new Instant(9)) + .advanceWatermarkTo(Instant.ofEpochMilli(9)) // If the timer fired, then this would case fn2 to fail with an assertion error. .addElements(KV.of("key", 1L)) .advanceWatermarkToInfinity(); @@ -5798,7 +5812,7 @@ public void processElement( // DoFn timer's watermark hold. This timer should not fire until the previous timer // fires and removes // the watermark hold. - timer.set(new Instant(8)); + timer.set(Instant.ofEpochMilli(8)); } @OnTimer(timerId) @@ -5818,7 +5832,7 @@ public void onTimer( // Normally this would case fn2's timer to expire, but it shouldn't here because of // the output timestamp. .advanceProcessingTime(Duration.standardSeconds(9)) - .advanceWatermarkTo(new Instant(11)) + .advanceWatermarkTo(Instant.ofEpochMilli(11)) // If the timer fired, then this would case fn2 to fail with an assertion error. .addElements(KV.of("key", 1L)) .advanceProcessingTime(Duration.standardSeconds(100)) @@ -5894,8 +5908,8 @@ public void onTimer( PCollection> input = pipeline.apply( TestStream.create(KvCoder.of(VoidCoder.of(), StringUtf8Coder.of())) - .addElements(TimestampedValue.of(KV.of(null, "foo"), new Instant(1))) - .addElements(TimestampedValue.of(KV.of(null, "bar"), new Instant(2))) + .addElements(TimestampedValue.of(KV.of(null, "foo"), Instant.ofEpochMilli(1))) + .addElements(TimestampedValue.of(KV.of(null, "bar"), Instant.ofEpochMilli(2))) .advanceWatermarkToInfinity()); PCollection result = input.apply(ParDo.of(bufferFn)); PAssert.that(result).containsInAnyOrder("foo", "bar"); @@ -6085,9 +6099,9 @@ public void onTimer(OutputReceiver r) { TestStream> stream = TestStream.create(KvCoder.of(StringUtf8Coder.of(), VarLongCoder.of())) - .advanceWatermarkTo(new Instant(0)) + .advanceWatermarkTo(Instant.ofEpochMilli(0)) .addElements(KV.of("hello", 37L)) - .advanceWatermarkTo(new Instant(0).plus(Duration.standardSeconds(1))) + .advanceWatermarkTo(Instant.ofEpochMilli(0).plus(Duration.standardSeconds(1))) .advanceWatermarkToInfinity(); PCollection output = pipeline.apply(stream).apply(ParDo.of(fn)); @@ -6163,9 +6177,9 @@ public void onTimer(OutputReceiver r) { TestStream> stream = TestStream.create(KvCoder.of(StringUtf8Coder.of(), VarLongCoder.of())) - .advanceWatermarkTo(new Instant(0)) + .advanceWatermarkTo(Instant.ofEpochMilli(0)) .addElements(KV.of("hello", 37L)) - .advanceWatermarkTo(new Instant(0).plus(Duration.standardSeconds(1))) + .advanceWatermarkTo(Instant.ofEpochMilli(0).plus(Duration.standardSeconds(1))) .advanceWatermarkToInfinity(); PCollection output = pipeline.apply(stream).apply(ParDo.of(fn)); @@ -6277,9 +6291,9 @@ public void clearTimer(OutputReceiver r) { TestStream> stream = TestStream.create(KvCoder.of(StringUtf8Coder.of(), VarLongCoder.of())) - .advanceWatermarkTo(new Instant(0)) + .advanceWatermarkTo(Instant.ofEpochMilli(0)) .addElements(KV.of("hello", 37L)) - .advanceWatermarkTo(new Instant(0).plus(Duration.standardSeconds(1))) + .advanceWatermarkTo(Instant.ofEpochMilli(0).plus(Duration.standardSeconds(1))) .advanceWatermarkToInfinity(); PCollection output = pipeline.apply(stream).apply(ParDo.of(fn)); @@ -6360,9 +6374,9 @@ public void onTimer(OutputReceiver r) { TestStream> stream = TestStream.create(KvCoder.of(StringUtf8Coder.of(), VarLongCoder.of())) - .advanceWatermarkTo(new Instant(0)) + .advanceWatermarkTo(Instant.ofEpochMilli(0)) .addElements(KV.of("hello", 37L), KV.of("hello", 38L)) - .advanceWatermarkTo(new Instant(0).plus(Duration.standardSeconds(1))) + .advanceWatermarkTo(Instant.ofEpochMilli(0).plus(Duration.standardSeconds(1))) .advanceWatermarkToInfinity(); PCollection output = pipeline.apply(stream).apply(ParDo.of(fn)); @@ -6776,8 +6790,8 @@ public void runTestTimerFamilyEventTime(boolean useStreaming) { @ProcessElement public void processElement( @TimerFamily(timerFamilyId) TimerMap timers, OutputReceiver r) { - timers.set("timer1", new Instant(1)); - timers.set("timer2", new Instant(2)); + timers.set("timer1", Instant.ofEpochMilli(1)); + timers.set("timer2", Instant.ofEpochMilli(2)); r.output("process"); } @@ -6835,8 +6849,8 @@ public void processElement( @TimerFamily(timerFamilyId1) TimerMap timerMap1, @TimerFamily(timerFamilyId2) TimerMap timerMap2, OutputReceiver r) { - timerMap1.set("timer", new Instant(1)); - timerMap2.set("timer", new Instant(2)); + timerMap1.set("timer", Instant.ofEpochMilli(1)); + timerMap2.set("timer", Instant.ofEpochMilli(2)); r.output("process"); } @@ -6897,8 +6911,8 @@ public void processElement( @TimerFamily(timerFamilyId) TimerMap timerMap, @TimerId(timerId) Timer timer, OutputReceiver r) { - timerMap.set("timer", new Instant(1)); - timer.set(new Instant(2)); + timerMap.set("timer", Instant.ofEpochMilli(1)); + timer.set(Instant.ofEpochMilli(2)); r.output("process"); } @@ -6990,7 +7004,7 @@ public void testKeyInOnTimer() throws Exception { @ProcessElement public void processElement(@TimerId(timerId) Timer timer, OutputReceiver r) { - timer.set(new Instant(1)); + timer.set(Instant.ofEpochMilli(1)); } @OnTimer(timerId) @@ -7022,7 +7036,7 @@ public void testKeyInOnTimerWithGenericKey() throws Exception { @ProcessElement public void processElement(@TimerId(timerId) Timer timer, OutputReceiver r) { - timer.set(new Instant(1)); + timer.set(Instant.ofEpochMilli(1)); } @OnTimer(timerId) @@ -7061,7 +7075,7 @@ public void testKeyInOnTimerWithWrongKeyType() throws Exception { @ProcessElement public void processElement(@TimerId(timerId) Timer timer, OutputReceiver r) { - timer.set(new Instant(1)); + timer.set(Instant.ofEpochMilli(1)); } @OnTimer(timerId) @@ -7092,7 +7106,7 @@ public void testKeyInOnTimerWithoutKV() throws Exception { @ProcessElement public void processElement(@TimerId(timerId) Timer timer, OutputReceiver r) { - timer.set(new Instant(1)); + timer.set(Instant.ofEpochMilli(1)); } @OnTimer(timerId) @@ -7156,8 +7170,10 @@ public void testOnWindowExpirationSimpleUnboundedGlobal() { public void runOnWindowExpirationSimple(boolean useStreaming, boolean globalWindow) { final String stateId = "foo"; final String timerId = "bar"; - IntervalWindow firstWindow = new IntervalWindow(new Instant(0), new Instant(10)); - IntervalWindow secondWindow = new IntervalWindow(new Instant(10), new Instant(20)); + IntervalWindow firstWindow = + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10)); + IntervalWindow secondWindow = + new IntervalWindow(Instant.ofEpochMilli(10), Instant.ofEpochMilli(20)); DoFn, Integer> fn = new DoFn, Integer>() { @@ -7171,7 +7187,7 @@ public void runOnWindowExpirationSimple(boolean useStreaming, boolean globalWind @ProcessElement public void processElement(@TimerId(timerId) Timer timer, BoundedWindow window) { - timer.set(new Instant(1)); + timer.set(Instant.ofEpochMilli(1)); } @OnTimer(timerId) @@ -7204,9 +7220,9 @@ public void onWindowExpiration( pipeline.apply( Create.timestamped( // first window - TimestampedValue.of(KV.of("hello", 7), new Instant(3)), + TimestampedValue.of(KV.of("hello", 7), Instant.ofEpochMilli(3)), // second window - TimestampedValue.of(KV.of("hi", 35), new Instant(13)))); + TimestampedValue.of(KV.of("hi", 35), Instant.ofEpochMilli(13)))); if (!globalWindow) { intermediate = intermediate.apply(Window.into(FixedWindows.of(Duration.millis(10)))); } @@ -7373,20 +7389,21 @@ public void finish(FinishBundleContext context) { }; int numBundles = 200; TestStream.Builder builder = - TestStream.create(StringUtf8Coder.of()).advanceWatermarkTo(new Instant(0)); + TestStream.create(StringUtf8Coder.of()).advanceWatermarkTo(Instant.ofEpochMilli(0)); List>> bundles = IntStream.range(0, numBundles) .mapToObj( r -> IntStream.range(0, r + 1) - .mapToObj(v -> TimestampedValue.of(String.valueOf(v), new Instant(r))) + .mapToObj( + v -> TimestampedValue.of(String.valueOf(v), Instant.ofEpochMilli(r))) .collect(Collectors.toList())) .collect(Collectors.toList()); for (List> b : bundles) { builder = builder .addElements(b.get(0), b.subList(1, b.size()).toArray(new TimestampedValue[] {})) - .advanceWatermarkTo(new Instant(b.size())); + .advanceWatermarkTo(Instant.ofEpochMilli(b.size())); } PCollection result = pipeline diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/RedistributeTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/RedistributeTest.java index ea46ffec4496..f9b968bde82b 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/RedistributeTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/RedistributeTest.java @@ -135,8 +135,8 @@ public void testRedistributePreservesTimestamps() { .apply( Create.timestamped( TimestampedValue.of("foo", BoundedWindow.TIMESTAMP_MIN_VALUE), - TimestampedValue.of("foo", new Instant(0)), - TimestampedValue.of("bar", new Instant(33)), + TimestampedValue.of("foo", Instant.ofEpochMilli(0)), + TimestampedValue.of("bar", Instant.ofEpochMilli(33)), TimestampedValue.of("bar", GlobalWindow.INSTANCE.maxTimestamp())) .withCoder(StringUtf8Coder.of())) .apply( @@ -187,12 +187,12 @@ public void testRedistributePreservesMetadata() { PaneInfo.NO_FIRING), WindowedValues.of( "foo", - new Instant(0), + Instant.ofEpochMilli(0), GlobalWindow.INSTANCE, PaneInfo.ON_TIME_AND_ONLY_FIRING), WindowedValues.of( "bar", - new Instant(33), + Instant.ofEpochMilli(33), GlobalWindow.INSTANCE, PaneInfo.createPane(false, false, PaneInfo.Timing.LATE, 1, 1)), WindowedValues.of( @@ -352,10 +352,10 @@ public void testRedistributeAfterSlidingWindows() { public void testRedistributeWithTimestampsStreaming() { TestStream stream = TestStream.create(VarLongCoder.of()) - .advanceWatermarkTo(new Instant(0L).plus(Duration.standardDays(48L))) + .advanceWatermarkTo(Instant.ofEpochMilli(0L).plus(Duration.standardDays(48L))) .addElements( - TimestampedValue.of(0L, new Instant(0L)), - TimestampedValue.of(1L, new Instant(0L).plus(Duration.standardDays(48L))), + TimestampedValue.of(0L, Instant.ofEpochMilli(0L)), + TimestampedValue.of(1L, Instant.ofEpochMilli(0L).plus(Duration.standardDays(48L))), TimestampedValue.of( 2L, BoundedWindow.TIMESTAMP_MAX_VALUE.minus(Duration.standardDays(48L)))) .advanceWatermarkToInfinity(); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ReifyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ReifyTest.java index 41c300a01a05..b4d4eac82eba 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ReifyTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ReifyTest.java @@ -44,7 +44,7 @@ @RunWith(JUnit4.class) public class ReifyTest implements Serializable { public static final WithTimestamps> TIMESTAMP_FROM_V = - WithTimestamps.of(input -> new Instant(input.getValue().longValue())); + WithTimestamps.of(input -> Instant.ofEpochMilli(input.getValue().longValue())); @Rule public transient TestPipeline pipeline = TestPipeline.create(); @Test @@ -53,10 +53,10 @@ public void extractFromValuesSucceeds() { PCollection>> preified = pipeline.apply( Create.of( - KV.of("foo", TimestampedValue.of(0, new Instant(0))), - KV.of("foo", TimestampedValue.of(1, new Instant(1))), - KV.of("bar", TimestampedValue.of(2, new Instant(2))), - KV.of("baz", TimestampedValue.of(3, new Instant(3))))); + KV.of("foo", TimestampedValue.of(0, Instant.ofEpochMilli(0))), + KV.of("foo", TimestampedValue.of(1, Instant.ofEpochMilli(1))), + KV.of("bar", TimestampedValue.of(2, Instant.ofEpochMilli(2))), + KV.of("baz", TimestampedValue.of(3, Instant.ofEpochMilli(3))))); PCollection> timestamped = preified.apply(Reify.extractTimestampsFromValues()); @@ -71,7 +71,7 @@ public void extractFromValuesSucceeds() { @ProcessElement public void verifyTimestampsEqualValue(ProcessContext context) { assertThat( - new Instant(context.element().getValue().longValue()), + Instant.ofEpochMilli(context.element().getValue().longValue()), equalTo(context.timestamp())); } })); @@ -86,13 +86,17 @@ public void extractFromValuesWhenValueTimestampedLaterSucceeds() { pipeline.apply( Create.timestamped( TimestampedValue.of( - KV.of("foo", TimestampedValue.of(0, new Instant(0))), new Instant(100)), + KV.of("foo", TimestampedValue.of(0, Instant.ofEpochMilli(0))), + Instant.ofEpochMilli(100)), TimestampedValue.of( - KV.of("foo", TimestampedValue.of(1, new Instant(1))), new Instant(101L)), + KV.of("foo", TimestampedValue.of(1, Instant.ofEpochMilli(1))), + Instant.ofEpochMilli(101L)), TimestampedValue.of( - KV.of("bar", TimestampedValue.of(2, new Instant(2))), new Instant(102L)), + KV.of("bar", TimestampedValue.of(2, Instant.ofEpochMilli(2))), + Instant.ofEpochMilli(102L)), TimestampedValue.of( - KV.of("baz", TimestampedValue.of(3, new Instant(3))), new Instant(103L)))); + KV.of("baz", TimestampedValue.of(3, Instant.ofEpochMilli(3))), + Instant.ofEpochMilli(103L)))); PCollection> timestamped = preified.apply(ReifyTimestamps.extractFromValues()); @@ -107,7 +111,7 @@ public void extractFromValuesWhenValueTimestampedLaterSucceeds() { @ProcessElement public void verifyTimestampsEqualValue(ProcessContext context) { assertThat( - new Instant(context.element().getValue().longValue()), + Instant.ofEpochMilli(context.element().getValue().longValue()), equalTo(context.timestamp())); } })); @@ -122,13 +126,13 @@ public void globalWindowNoKeys() { pipeline .apply( TestStream.create(StringUtf8Coder.of()) - .addElements(TimestampedValue.of("dei", new Instant(123L))) + .addElements(TimestampedValue.of("dei", Instant.ofEpochMilli(123L))) .advanceWatermarkToInfinity()) .apply(Reify.windows()); PAssert.that(result) .containsInAnyOrder( ValueInSingleWindow.of( - "dei", new Instant(123L), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); + "dei", Instant.ofEpochMilli(123L), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); pipeline.run(); } @@ -145,10 +149,10 @@ public void timestampedValuesSucceeds() { PAssert.that(reified) .containsInAnyOrder( - KV.of("foo", TimestampedValue.of(0, new Instant(0))), - KV.of("foo", TimestampedValue.of(1, new Instant(1))), - KV.of("bar", TimestampedValue.of(2, new Instant(2))), - KV.of("baz", TimestampedValue.of(3, new Instant(3)))); + KV.of("foo", TimestampedValue.of(0, Instant.ofEpochMilli(0))), + KV.of("foo", TimestampedValue.of(1, Instant.ofEpochMilli(1))), + KV.of("bar", TimestampedValue.of(2, Instant.ofEpochMilli(2))), + KV.of("baz", TimestampedValue.of(3, Instant.ofEpochMilli(3)))); pipeline.run(); } @@ -159,14 +163,15 @@ public void timestampsSucceeds() { PCollection timestamped = pipeline.apply( Create.timestamped( - TimestampedValue.of("foo", new Instant(0L)), - TimestampedValue.of("bar", new Instant(1L)))); + TimestampedValue.of("foo", Instant.ofEpochMilli(0L)), + TimestampedValue.of("bar", Instant.ofEpochMilli(1L)))); PCollection> reified = timestamped.apply(Reify.timestamps()); PAssert.that(reified) .containsInAnyOrder( - TimestampedValue.of("foo", new Instant(0)), TimestampedValue.of("bar", new Instant(1))); + TimestampedValue.of("foo", Instant.ofEpochMilli(0)), + TimestampedValue.of("bar", Instant.ofEpochMilli(1))); pipeline.run(); } @@ -187,19 +192,19 @@ public void windowsInValueSucceeds() { KV.of( "foo", ValueInSingleWindow.of( - 0, new Instant(0), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)), + 0, Instant.ofEpochMilli(0), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)), KV.of( "foo", ValueInSingleWindow.of( - 1, new Instant(1), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)), + 1, Instant.ofEpochMilli(1), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)), KV.of( "bar", ValueInSingleWindow.of( - 2, new Instant(2), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)), + 2, Instant.ofEpochMilli(2), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)), KV.of( "baz", ValueInSingleWindow.of( - 3, new Instant(3), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING))); + 3, Instant.ofEpochMilli(3), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING))); pipeline.run(); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ReifyTimestampsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ReifyTimestampsTest.java index c76fb72325f7..6baa42ee16bc 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ReifyTimestampsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ReifyTimestampsTest.java @@ -45,17 +45,17 @@ public void inValuesSucceeds() { PCollection> timestamped = pipeline .apply(Create.of(KV.of("foo", 0), KV.of("foo", 1), KV.of("bar", 2), KV.of("baz", 3))) - .apply(WithTimestamps.of(input -> new Instant(input.getValue().longValue()))); + .apply(WithTimestamps.of(input -> Instant.ofEpochMilli(input.getValue().longValue()))); PCollection>> reified = timestamped.apply(ReifyTimestamps.inValues()); PAssert.that(reified) .containsInAnyOrder( - KV.of("foo", TimestampedValue.of(0, new Instant(0))), - KV.of("foo", TimestampedValue.of(1, new Instant(1))), - KV.of("bar", TimestampedValue.of(2, new Instant(2))), - KV.of("baz", TimestampedValue.of(3, new Instant(3)))); + KV.of("foo", TimestampedValue.of(0, Instant.ofEpochMilli(0))), + KV.of("foo", TimestampedValue.of(1, Instant.ofEpochMilli(1))), + KV.of("bar", TimestampedValue.of(2, Instant.ofEpochMilli(2))), + KV.of("baz", TimestampedValue.of(3, Instant.ofEpochMilli(3)))); pipeline.run(); } @@ -66,10 +66,10 @@ public void extractFromValuesSucceeds() { PCollection>> preified = pipeline.apply( Create.of( - KV.of("foo", TimestampedValue.of(0, new Instant(0))), - KV.of("foo", TimestampedValue.of(1, new Instant(1))), - KV.of("bar", TimestampedValue.of(2, new Instant(2))), - KV.of("baz", TimestampedValue.of(3, new Instant(3))))); + KV.of("foo", TimestampedValue.of(0, Instant.ofEpochMilli(0))), + KV.of("foo", TimestampedValue.of(1, Instant.ofEpochMilli(1))), + KV.of("bar", TimestampedValue.of(2, Instant.ofEpochMilli(2))), + KV.of("baz", TimestampedValue.of(3, Instant.ofEpochMilli(3))))); PCollection> timestamped = preified.apply(ReifyTimestamps.extractFromValues()); @@ -84,7 +84,7 @@ public void extractFromValuesSucceeds() { @ProcessElement public void verifyTimestampsEqualValue(ProcessContext context) { assertThat( - new Instant(context.element().getValue().longValue()), + Instant.ofEpochMilli(context.element().getValue().longValue()), equalTo(context.timestamp())); } })); @@ -99,13 +99,17 @@ public void extractFromValuesWhenValueTimestampedLaterSucceeds() { pipeline.apply( Create.timestamped( TimestampedValue.of( - KV.of("foo", TimestampedValue.of(0, new Instant(0))), new Instant(100)), + KV.of("foo", TimestampedValue.of(0, Instant.ofEpochMilli(0))), + Instant.ofEpochMilli(100)), TimestampedValue.of( - KV.of("foo", TimestampedValue.of(1, new Instant(1))), new Instant(101L)), + KV.of("foo", TimestampedValue.of(1, Instant.ofEpochMilli(1))), + Instant.ofEpochMilli(101L)), TimestampedValue.of( - KV.of("bar", TimestampedValue.of(2, new Instant(2))), new Instant(102L)), + KV.of("bar", TimestampedValue.of(2, Instant.ofEpochMilli(2))), + Instant.ofEpochMilli(102L)), TimestampedValue.of( - KV.of("baz", TimestampedValue.of(3, new Instant(3))), new Instant(103L)))); + KV.of("baz", TimestampedValue.of(3, Instant.ofEpochMilli(3))), + Instant.ofEpochMilli(103L)))); PCollection> timestamped = preified.apply(ReifyTimestamps.extractFromValues()); @@ -120,7 +124,7 @@ public void extractFromValuesWhenValueTimestampedLaterSucceeds() { @ProcessElement public void verifyTimestampsEqualValue(ProcessContext context) { assertThat( - new Instant(context.element().getValue().longValue()), + Instant.ofEpochMilli(context.element().getValue().longValue()), equalTo(context.timestamp())); } })); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ReshuffleTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ReshuffleTest.java index 10f65dfce68d..126843e1b88f 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ReshuffleTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ReshuffleTest.java @@ -126,8 +126,8 @@ public void testReshufflePreservesTimestamps() { .apply( Create.timestamped( TimestampedValue.of("foo", BoundedWindow.TIMESTAMP_MIN_VALUE), - TimestampedValue.of("foo", new Instant(0)), - TimestampedValue.of("bar", new Instant(33)), + TimestampedValue.of("foo", Instant.ofEpochMilli(0)), + TimestampedValue.of("bar", Instant.ofEpochMilli(33)), TimestampedValue.of("bar", GlobalWindow.INSTANCE.maxTimestamp())) .withCoder(StringUtf8Coder.of())) .apply( @@ -178,12 +178,12 @@ public void testReshufflePreservesMetadata() { PaneInfo.NO_FIRING), WindowedValues.of( "foo", - new Instant(0), + Instant.ofEpochMilli(0), GlobalWindow.INSTANCE, PaneInfo.ON_TIME_AND_ONLY_FIRING), WindowedValues.of( "bar", - new Instant(33), + Instant.ofEpochMilli(33), GlobalWindow.INSTANCE, PaneInfo.createPane(false, false, PaneInfo.Timing.LATE, 1, 1)), WindowedValues.of( @@ -343,10 +343,10 @@ public void testReshuffleAfterSlidingWindows() { public void testReshuffleWithTimestampsStreaming() { TestStream stream = TestStream.create(VarLongCoder.of()) - .advanceWatermarkTo(new Instant(0L).plus(Duration.standardDays(48L))) + .advanceWatermarkTo(Instant.ofEpochMilli(0L).plus(Duration.standardDays(48L))) .addElements( - TimestampedValue.of(0L, new Instant(0L)), - TimestampedValue.of(1L, new Instant(0L).plus(Duration.standardDays(48L))), + TimestampedValue.of(0L, Instant.ofEpochMilli(0L)), + TimestampedValue.of(1L, Instant.ofEpochMilli(0L).plus(Duration.standardDays(48L))), TimestampedValue.of( 2L, BoundedWindow.TIMESTAMP_MAX_VALUE.minus(Duration.standardDays(48L)))) .advanceWatermarkToInfinity(); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SampleTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SampleTest.java index 615742934921..00d2e96799d0 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SampleTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SampleTest.java @@ -215,7 +215,7 @@ public Void apply(Iterable in) { } private static TimestampedValue tv(int i) { - return TimestampedValue.of(i, new Instant(i * 1000)); + return TimestampedValue.of(i, Instant.ofEpochMilli(i * 1000)); } @Test @@ -230,10 +230,10 @@ public void testSampleAny() { PCollection output = input.apply(Sample.any(2)); PAssert.that(output) - .inWindow(new IntervalWindow(new Instant(0), Duration.standardSeconds(3))) + .inWindow(new IntervalWindow(Instant.ofEpochMilli(0), Duration.standardSeconds(3))) .satisfies(new VerifyCorrectSample<>(2, Arrays.asList(0, 1, 2))); PAssert.that(output) - .inWindow(new IntervalWindow(new Instant(3000), Duration.standardSeconds(3))) + .inWindow(new IntervalWindow(Instant.ofEpochMilli(3000), Duration.standardSeconds(3))) .satisfies(new VerifyCorrectSample<>(2, Arrays.asList(3, 4, 5))); pipeline.run(); } @@ -264,10 +264,10 @@ public void testSampleAnyZero() { .apply(Sample.any(0)); PAssert.that(output) - .inWindow(new IntervalWindow(new Instant(0), Duration.standardSeconds(3))) + .inWindow(new IntervalWindow(Instant.ofEpochMilli(0), Duration.standardSeconds(3))) .satisfies(new VerifyCorrectSample<>(0, EMPTY)); PAssert.that(output) - .inWindow(new IntervalWindow(new Instant(3000), Duration.standardSeconds(3))) + .inWindow(new IntervalWindow(Instant.ofEpochMilli(3000), Duration.standardSeconds(3))) .satisfies(new VerifyCorrectSample<>(0, EMPTY)); pipeline.run(); } @@ -282,7 +282,7 @@ public void testSampleAnyInsufficientElements() { .apply(Sample.any(10)); PAssert.that(output) - .inWindow(new IntervalWindow(new Instant(0), Duration.standardSeconds(3))) + .inWindow(new IntervalWindow(Instant.ofEpochMilli(0), Duration.standardSeconds(3))) .satisfies(new VerifyCorrectSample<>(0, EMPTY)); pipeline.run(); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SimpleStatsFnsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SimpleStatsFnsTest.java index b04beb47b49e..1379a02c42b7 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SimpleStatsFnsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SimpleStatsFnsTest.java @@ -87,18 +87,22 @@ public TestCase(NumT min, NumT max, NumT sum, NumT... values) { @Test public void testInstantStats() { assertEquals( - new Instant(1000), - Min.naturalOrder().apply(Arrays.asList(new Instant(1000), new Instant(2000)))); + Instant.ofEpochMilli(1000), + Min.naturalOrder() + .apply(Arrays.asList(Instant.ofEpochMilli(1000), Instant.ofEpochMilli(2000)))); assertEquals(null, Min.naturalOrder().apply(Collections.emptyList())); assertEquals( - new Instant(5000), Min.naturalOrder(new Instant(5000)).apply(Collections.emptyList())); + Instant.ofEpochMilli(5000), + Min.naturalOrder(Instant.ofEpochMilli(5000)).apply(Collections.emptyList())); assertEquals( - new Instant(2000), - Max.naturalOrder().apply(Arrays.asList(new Instant(1000), new Instant(2000)))); + Instant.ofEpochMilli(2000), + Max.naturalOrder() + .apply(Arrays.asList(Instant.ofEpochMilli(1000), Instant.ofEpochMilli(2000)))); assertEquals(null, Max.naturalOrder().apply(Collections.emptyList())); assertEquals( - new Instant(5000), Max.naturalOrder(new Instant(5000)).apply(Collections.emptyList())); + Instant.ofEpochMilli(5000), + Max.naturalOrder(Instant.ofEpochMilli(5000)).apply(Collections.emptyList())); } @Test diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SplittableDoFnTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SplittableDoFnTest.java index 8ce9330b9ab4..b44599dcdadd 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SplittableDoFnTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SplittableDoFnTest.java @@ -475,22 +475,22 @@ private void testWindowedSideInput(IsBounded bounded) { p.apply( "main", Create.timestamped( - TimestampedValue.of(0, new Instant(0)), - TimestampedValue.of(1, new Instant(1)), - TimestampedValue.of(2, new Instant(2)), - TimestampedValue.of(3, new Instant(3)), - TimestampedValue.of(4, new Instant(4)), - TimestampedValue.of(5, new Instant(5)), - TimestampedValue.of(6, new Instant(6)), - TimestampedValue.of(7, new Instant(7)))) + TimestampedValue.of(0, Instant.ofEpochMilli(0)), + TimestampedValue.of(1, Instant.ofEpochMilli(1)), + TimestampedValue.of(2, Instant.ofEpochMilli(2)), + TimestampedValue.of(3, Instant.ofEpochMilli(3)), + TimestampedValue.of(4, Instant.ofEpochMilli(4)), + TimestampedValue.of(5, Instant.ofEpochMilli(5)), + TimestampedValue.of(6, Instant.ofEpochMilli(6)), + TimestampedValue.of(7, Instant.ofEpochMilli(7)))) .apply("window 2", Window.into(FixedWindows.of(Duration.millis(2)))); PCollectionView sideInput = p.apply( "side", Create.timestamped( - TimestampedValue.of("a", new Instant(0)), - TimestampedValue.of("b", new Instant(4)))) + TimestampedValue.of("a", Instant.ofEpochMilli(0)), + TimestampedValue.of("b", Instant.ofEpochMilli(4)))) .apply("window 4", Window.into(FixedWindows.of(Duration.millis(4)))) .apply("singleton", View.asSingleton()); @@ -500,14 +500,14 @@ private void testWindowedSideInput(IsBounded bounded) { sdfWithSideInput( bounded, ImmutableMap.builder() - .put(new Instant(0), "a") - .put(new Instant(1), "a") - .put(new Instant(2), "a") - .put(new Instant(3), "a") - .put(new Instant(4), "b") - .put(new Instant(5), "b") - .put(new Instant(6), "b") - .put(new Instant(7), "b") + .put(Instant.ofEpochMilli(0), "a") + .put(Instant.ofEpochMilli(1), "a") + .put(Instant.ofEpochMilli(2), "a") + .put(Instant.ofEpochMilli(3), "a") + .put(Instant.ofEpochMilli(4), "b") + .put(Instant.ofEpochMilli(5), "b") + .put(Instant.ofEpochMilli(6), "b") + .put(Instant.ofEpochMilli(7), "b") .build())) .withSideInput("sideInput", sideInput)); @@ -667,18 +667,18 @@ private void testWindowedSideInputWithCheckpoints(IsBounded bounded) { p.apply( "main", Create.timestamped( - TimestampedValue.of(0, new Instant(0)), - TimestampedValue.of(1, new Instant(1)), - TimestampedValue.of(2, new Instant(2)), - TimestampedValue.of(3, new Instant(3)))) + TimestampedValue.of(0, Instant.ofEpochMilli(0)), + TimestampedValue.of(1, Instant.ofEpochMilli(1)), + TimestampedValue.of(2, Instant.ofEpochMilli(2)), + TimestampedValue.of(3, Instant.ofEpochMilli(3)))) .apply("window 1", Window.into(FixedWindows.of(Duration.millis(1)))); PCollectionView sideInput = p.apply( "side", Create.timestamped( - TimestampedValue.of("a", new Instant(0)), - TimestampedValue.of("b", new Instant(2)))) + TimestampedValue.of("a", Instant.ofEpochMilli(0)), + TimestampedValue.of("b", Instant.ofEpochMilli(2)))) .apply("window 2", Window.into(FixedWindows.of(Duration.millis(2)))) .apply("singleton", View.asSingleton()); @@ -688,10 +688,10 @@ private void testWindowedSideInputWithCheckpoints(IsBounded bounded) { sdfWithMultipleOutputsPerBlockAndSideInput( bounded, ImmutableMap.builder() - .put(new Instant(0), "a") - .put(new Instant(1), "a") - .put(new Instant(2), "b") - .put(new Instant(3), "b") + .put(Instant.ofEpochMilli(0), "a") + .put(Instant.ofEpochMilli(1), "a") + .put(Instant.ofEpochMilli(2), "b") + .put(Instant.ofEpochMilli(3), "b") .build(), 3 /* numClaimsPerCall */)) .withSideInput("sideInput", sideInput)); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java index 06aa9adaf745..7c6e0ec12a9a 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java @@ -118,8 +118,8 @@ public void testWindowedSingletonSideInput() { .apply( "Create47", Create.timestamped( - TimestampedValue.of(47, new Instant(1)), - TimestampedValue.of(48, new Instant(11)))) + TimestampedValue.of(47, Instant.ofEpochMilli(1)), + TimestampedValue.of(48, Instant.ofEpochMilli(11)))) .apply("SideWindowInto", Window.into(FixedWindows.of(Duration.millis(10)))) .apply(View.asSingleton()); @@ -128,9 +128,9 @@ public void testWindowedSingletonSideInput() { .apply( "Create123", Create.timestamped( - TimestampedValue.of(1, new Instant(4)), - TimestampedValue.of(2, new Instant(8)), - TimestampedValue.of(3, new Instant(12)))) + TimestampedValue.of(1, Instant.ofEpochMilli(4)), + TimestampedValue.of(2, Instant.ofEpochMilli(8)), + TimestampedValue.of(3, Instant.ofEpochMilli(12)))) .apply("MainWindowInto", Window.into(FixedWindows.of(Duration.millis(10)))) .apply( "OutputSideInputs", @@ -154,9 +154,9 @@ public void testWindowedSideInputNotPresent() { PCollection> input = pipeline.apply( TestStream.create(KvCoder.of(VarLongCoder.of(), VarLongCoder.of())) - .advanceWatermarkTo(new Instant(0)) - .addElements(TimestampedValue.of(KV.of(1000L, 1000L), new Instant(1000L))) - .advanceWatermarkTo(new Instant(20000)) + .advanceWatermarkTo(Instant.ofEpochMilli(0)) + .addElements(TimestampedValue.of(KV.of(1000L, 1000L), Instant.ofEpochMilli(1000L))) + .advanceWatermarkTo(Instant.ofEpochMilli(20000)) .advanceWatermarkToInfinity()); final PCollectionView view = @@ -183,7 +183,7 @@ public void processElement(ProcessContext c) { .withSideInputs(view)); PAssert.that(output) - .inWindow(new IntervalWindow(new Instant(0), new Instant(10000))) + .inWindow(new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10000))) .containsInAnyOrder(0L); pipeline.run(); @@ -271,7 +271,8 @@ public void processElement(ProcessContext c) { @Test @Category({ValidatesRunner.class, UsesTriggeredSideInputs.class}) public void testTriggeredLatestSingleton() { - IntervalWindow zeroWindow = new IntervalWindow(new Instant(0), new Instant(1000)); + IntervalWindow zeroWindow = + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(1000)); PCollectionView view = pipeline @@ -293,7 +294,7 @@ public void testTriggeredLatestSingleton() { PCollection pc = pipeline .apply(Impulse.create()) - .apply(WithTimestamps.of(impulse -> new Instant(0))) + .apply(WithTimestamps.of(impulse -> Instant.ofEpochMilli(0))) .apply("Window main input", Window.into(FixedWindows.of(Duration.standardSeconds(1)))) .apply( ParDo.of( @@ -426,14 +427,14 @@ public void testWindowedListSideInput() { .apply( "CreateSideInput", Create.timestamped( - TimestampedValue.of(11, new Instant(1)), - TimestampedValue.of(13, new Instant(1)), - TimestampedValue.of(17, new Instant(1)), - TimestampedValue.of(23, new Instant(1)), - TimestampedValue.of(31, new Instant(11)), - TimestampedValue.of(33, new Instant(11)), - TimestampedValue.of(37, new Instant(11)), - TimestampedValue.of(43, new Instant(11)))) + TimestampedValue.of(11, Instant.ofEpochMilli(1)), + TimestampedValue.of(13, Instant.ofEpochMilli(1)), + TimestampedValue.of(17, Instant.ofEpochMilli(1)), + TimestampedValue.of(23, Instant.ofEpochMilli(1)), + TimestampedValue.of(31, Instant.ofEpochMilli(11)), + TimestampedValue.of(33, Instant.ofEpochMilli(11)), + TimestampedValue.of(37, Instant.ofEpochMilli(11)), + TimestampedValue.of(43, Instant.ofEpochMilli(11)))) .apply("SideWindowInto", Window.into(FixedWindows.of(Duration.millis(10)))) .apply(View.asList()); @@ -442,8 +443,8 @@ public void testWindowedListSideInput() { .apply( "CreateMainInput", Create.timestamped( - TimestampedValue.of(29, new Instant(1)), - TimestampedValue.of(35, new Instant(11)))) + TimestampedValue.of(29, Instant.ofEpochMilli(1)), + TimestampedValue.of(35, Instant.ofEpochMilli(11)))) .apply("MainWindowInto", Window.into(FixedWindows.of(Duration.millis(10)))) .apply( "OutputSideInputs", @@ -581,14 +582,14 @@ public void testWindowedIterableSideInput() { .apply( "CreateSideInput", Create.timestamped( - TimestampedValue.of(11, new Instant(1)), - TimestampedValue.of(13, new Instant(1)), - TimestampedValue.of(17, new Instant(1)), - TimestampedValue.of(23, new Instant(1)), - TimestampedValue.of(31, new Instant(11)), - TimestampedValue.of(33, new Instant(11)), - TimestampedValue.of(37, new Instant(11)), - TimestampedValue.of(43, new Instant(11)))) + TimestampedValue.of(11, Instant.ofEpochMilli(1)), + TimestampedValue.of(13, Instant.ofEpochMilli(1)), + TimestampedValue.of(17, Instant.ofEpochMilli(1)), + TimestampedValue.of(23, Instant.ofEpochMilli(1)), + TimestampedValue.of(31, Instant.ofEpochMilli(11)), + TimestampedValue.of(33, Instant.ofEpochMilli(11)), + TimestampedValue.of(37, Instant.ofEpochMilli(11)), + TimestampedValue.of(43, Instant.ofEpochMilli(11)))) .apply("SideWindowInto", Window.into(FixedWindows.of(Duration.millis(10)))) .apply(View.asIterable()); @@ -597,8 +598,8 @@ public void testWindowedIterableSideInput() { .apply( "CreateMainInput", Create.timestamped( - TimestampedValue.of(29, new Instant(1)), - TimestampedValue.of(35, new Instant(11)))) + TimestampedValue.of(29, Instant.ofEpochMilli(1)), + TimestampedValue.of(35, Instant.ofEpochMilli(11)))) .apply("MainWindowInto", Window.into(FixedWindows.of(Duration.millis(10)))) .apply( "OutputSideInputs", @@ -690,9 +691,9 @@ public void testWindowedSideInputFixedToFixed() { .apply( "CreateSideInput", Create.timestamped( - TimestampedValue.of(1, new Instant(1)), - TimestampedValue.of(2, new Instant(11)), - TimestampedValue.of(3, new Instant(13)))) + TimestampedValue.of(1, Instant.ofEpochMilli(1)), + TimestampedValue.of(2, Instant.ofEpochMilli(11)), + TimestampedValue.of(3, Instant.ofEpochMilli(13)))) .apply("WindowSideInput", Window.into(FixedWindows.of(Duration.millis(10)))) .apply(Sum.integersGlobally().withoutDefaults()) .apply(View.asSingleton()); @@ -702,9 +703,9 @@ public void testWindowedSideInputFixedToFixed() { .apply( "CreateMainInput", Create.timestamped( - TimestampedValue.of("A", new Instant(4)), - TimestampedValue.of("B", new Instant(15)), - TimestampedValue.of("C", new Instant(7)))) + TimestampedValue.of("A", Instant.ofEpochMilli(4)), + TimestampedValue.of("B", Instant.ofEpochMilli(15)), + TimestampedValue.of("C", Instant.ofEpochMilli(7)))) .apply("WindowMainInput", Window.into(FixedWindows.of(Duration.millis(10)))) .apply( "OutputMainAndSideInputs", @@ -731,9 +732,9 @@ public void testWindowedSideInputFixedToGlobal() { .apply( "CreateSideInput", Create.timestamped( - TimestampedValue.of(1, new Instant(1)), - TimestampedValue.of(2, new Instant(11)), - TimestampedValue.of(3, new Instant(13)))) + TimestampedValue.of(1, Instant.ofEpochMilli(1)), + TimestampedValue.of(2, Instant.ofEpochMilli(11)), + TimestampedValue.of(3, Instant.ofEpochMilli(13)))) .apply("WindowSideInput", Window.into(new GlobalWindows())) .apply(Sum.integersGlobally()) .apply(View.asSingleton()); @@ -743,9 +744,9 @@ public void testWindowedSideInputFixedToGlobal() { .apply( "CreateMainInput", Create.timestamped( - TimestampedValue.of("A", new Instant(4)), - TimestampedValue.of("B", new Instant(15)), - TimestampedValue.of("C", new Instant(7)))) + TimestampedValue.of("A", Instant.ofEpochMilli(4)), + TimestampedValue.of("B", Instant.ofEpochMilli(15)), + TimestampedValue.of("C", Instant.ofEpochMilli(7)))) .apply("WindowMainInput", Window.into(FixedWindows.of(Duration.millis(10)))) .apply( "OutputMainAndSideInputs", @@ -772,8 +773,8 @@ public void testWindowedSideInputFixedToFixedWithDefault() { .apply( "CreateSideInput", Create.timestamped( - TimestampedValue.of(2, new Instant(11)), - TimestampedValue.of(3, new Instant(13)))) + TimestampedValue.of(2, Instant.ofEpochMilli(11)), + TimestampedValue.of(3, Instant.ofEpochMilli(13)))) .apply("WindowSideInput", Window.into(FixedWindows.of(Duration.millis(10)))) .apply(Sum.integersGlobally().asSingletonView()); @@ -782,9 +783,9 @@ public void testWindowedSideInputFixedToFixedWithDefault() { .apply( "CreateMainInput", Create.timestamped( - TimestampedValue.of("A", new Instant(4)), - TimestampedValue.of("B", new Instant(15)), - TimestampedValue.of("C", new Instant(7)))) + TimestampedValue.of("A", Instant.ofEpochMilli(4)), + TimestampedValue.of("B", Instant.ofEpochMilli(15)), + TimestampedValue.of("C", Instant.ofEpochMilli(7)))) .apply("WindowMainInput", Window.into(FixedWindows.of(Duration.millis(10)))) .apply( "OutputMainAndSideInputs", diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithTimestampsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithTimestampsTest.java index 5b8eb6c256e8..f4efa2fc400e 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithTimestampsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithTimestampsTest.java @@ -51,7 +51,8 @@ public class WithTimestampsTest implements Serializable { @Category(ValidatesRunner.class) public void withTimestampsShouldApplyTimestamps() { - SerializableFunction timestampFn = input -> new Instant(Long.valueOf(input)); + SerializableFunction timestampFn = + input -> Instant.ofEpochMilli(Long.valueOf(input)); String yearTwoThousand = "946684800000"; PCollection timestamped = @@ -73,10 +74,10 @@ public void processElement(DoFn>.ProcessContext c) .containsInAnyOrder(yearTwoThousand, "0", "1234", Integer.toString(Integer.MAX_VALUE)); PAssert.that(timestampedVals) .containsInAnyOrder( - KV.of("0", new Instant(0)), - KV.of("1234", new Instant(1234L)), - KV.of(Integer.toString(Integer.MAX_VALUE), new Instant(Integer.MAX_VALUE)), - KV.of(yearTwoThousand, new Instant(Long.valueOf(yearTwoThousand)))); + KV.of("0", Instant.ofEpochMilli(0)), + KV.of("1234", Instant.ofEpochMilli(1234L)), + KV.of(Integer.toString(Integer.MAX_VALUE), Instant.ofEpochMilli(Integer.MAX_VALUE)), + KV.of(yearTwoThousand, Instant.ofEpochMilli(Long.valueOf(yearTwoThousand)))); p.run(); } @@ -85,9 +86,10 @@ public void processElement(DoFn>.ProcessContext c) @Category(NeedsRunner.class) public void withTimestampsBackwardsInTimeShouldThrow() { - SerializableFunction timestampFn = input -> new Instant(Long.valueOf(input)); + SerializableFunction timestampFn = + input -> Instant.ofEpochMilli(Long.valueOf(input)); SerializableFunction backInTimeFn = - input -> new Instant(Long.valueOf(input)).minus(Duration.millis(1000L)); + input -> Instant.ofEpochMilli(Long.valueOf(input)).minus(Duration.millis(1000L)); String yearTwoThousand = "946684800000"; @@ -106,11 +108,12 @@ public void withTimestampsBackwardsInTimeShouldThrow() { @Category(ValidatesRunner.class) public void withTimestampsBackwardsInTimeAndWithAllowedTimestampSkewShouldSucceed() { - SerializableFunction timestampFn = input -> new Instant(Long.valueOf(input)); + SerializableFunction timestampFn = + input -> Instant.ofEpochMilli(Long.valueOf(input)); final Duration skew = Duration.millis(1000L); SerializableFunction backInTimeFn = - input -> new Instant(Long.valueOf(input)).minus(skew); + input -> Instant.ofEpochMilli(Long.valueOf(input)).minus(skew); String yearTwoThousand = "946684800000"; PCollection timestampedWithSkew = @@ -136,12 +139,13 @@ public void processElement(DoFn>.ProcessContext c) .containsInAnyOrder(yearTwoThousand, "0", "1234", Integer.toString(Integer.MAX_VALUE)); PAssert.that(timestampedVals) .containsInAnyOrder( - KV.of("0", new Instant(0L).minus(skew)), - KV.of("1234", new Instant(1234L).minus(skew)), + KV.of("0", Instant.ofEpochMilli(0L).minus(skew)), + KV.of("1234", Instant.ofEpochMilli(1234L).minus(skew)), KV.of( Integer.toString(Integer.MAX_VALUE), - new Instant(Long.valueOf(Integer.MAX_VALUE)).minus(skew)), - KV.of(yearTwoThousand, new Instant(Long.valueOf(yearTwoThousand)).minus(skew))); + Instant.ofEpochMilli(Long.valueOf(Integer.MAX_VALUE)).minus(skew)), + KV.of( + yearTwoThousand, Instant.ofEpochMilli(Long.valueOf(yearTwoThousand)).minus(skew))); p.run(); } @@ -185,7 +189,7 @@ public void withTimestampsLambdaShouldApplyTimestamps() { final String yearTwoThousand = "946684800000"; PCollection timestamped = p.apply(Create.of("1234", "0", Integer.toString(Integer.MAX_VALUE), yearTwoThousand)) - .apply(WithTimestamps.of((String input) -> new Instant(Long.valueOf(input)))); + .apply(WithTimestamps.of((String input) -> Instant.ofEpochMilli(Long.valueOf(input)))); PCollection> timestampedVals = timestamped.apply( @@ -201,10 +205,10 @@ public void processElement(ProcessContext c) throws Exception { .containsInAnyOrder(yearTwoThousand, "0", "1234", Integer.toString(Integer.MAX_VALUE)); PAssert.that(timestampedVals) .containsInAnyOrder( - KV.of("0", new Instant(0)), - KV.of("1234", new Instant(Long.valueOf("1234"))), - KV.of(Integer.toString(Integer.MAX_VALUE), new Instant(Integer.MAX_VALUE)), - KV.of(yearTwoThousand, new Instant(Long.valueOf(yearTwoThousand)))); + KV.of("0", Instant.ofEpochMilli(0)), + KV.of("1234", Instant.ofEpochMilli(Long.valueOf("1234"))), + KV.of(Integer.toString(Integer.MAX_VALUE), Instant.ofEpochMilli(Integer.MAX_VALUE)), + KV.of(yearTwoThousand, Instant.ofEpochMilli(Long.valueOf(yearTwoThousand)))); p.run(); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java index c80e92a897f9..6b56c0e140ad 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java @@ -106,7 +106,7 @@ public void populateDisplayData(DisplayData.Builder builder) { PTransform transform = new PTransform, PCollection>() { - final Instant defaultStartTime = new Instant(0); + final Instant defaultStartTime = Instant.ofEpochMilli(0); Instant startTime = defaultStartTime; @Override @@ -337,9 +337,11 @@ public void populateDisplayData(DisplayData.Builder builder) { .addIfNotDefault(DisplayData.item("defaultBoolean", true), true) .addIfNotDefault(DisplayData.item("notDefaultBoolean", true), false) .addIfNotDefault( - DisplayData.item("defaultInstant", new Instant(0)), new Instant(0)) + DisplayData.item("defaultInstant", Instant.ofEpochMilli(0)), + Instant.ofEpochMilli(0)) .addIfNotDefault( - DisplayData.item("notDefaultInstant", new Instant(0)), Instant.now()) + DisplayData.item("notDefaultInstant", Instant.ofEpochMilli(0)), + Instant.now()) .addIfNotDefault( DisplayData.item("defaultDuration", Duration.ZERO), Duration.ZERO) .addIfNotDefault( @@ -1153,7 +1155,7 @@ public void populateDisplayData(DisplayData.Builder builder) { .add(DisplayData.item("long", intValue)) .add(DisplayData.item("double", floatValue)) .add(DisplayData.item("boolean", boolValue)) - .add(DisplayData.item("instant", new Instant(0))) + .add(DisplayData.item("instant", Instant.ofEpochMilli(0))) .add(DisplayData.item("duration", Duration.millis(durationMillis))) .add( DisplayData.item("class", DisplayDataTest.class) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java index 186d58e33189..66bfff21528a 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java @@ -120,7 +120,7 @@ public class DoFnInvokersTest { @Before public void setUp() { mockElement = "element"; - mockTimestamp = new Instant(0); + mockTimestamp = Instant.ofEpochMilli(0); MockitoAnnotations.initMocks(this); when(mockArgumentProvider.window()).thenReturn(mockWindow); // when(mockArgumentProvider.paneInfo(Matchers.any())) @@ -1350,7 +1350,8 @@ public void onMyTimer() { @Test public void testOnTimerWithWindow() throws Exception { final String timerId = "my-timer-id"; - final IntervalWindow testWindow = new IntervalWindow(new Instant(0), new Instant(15)); + final IntervalWindow testWindow = + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(15)); when(mockArgumentProvider.window()).thenReturn(testWindow); class SimpleTimerDoFn extends DoFn { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/splittabledofn/WatermarkEstimatorsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/splittabledofn/WatermarkEstimatorsTest.java index b028f4d95eae..1a85c6bd7412 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/splittabledofn/WatermarkEstimatorsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/splittabledofn/WatermarkEstimatorsTest.java @@ -62,7 +62,7 @@ public void testWallTimeWatermarkEstimator() { try { DateTimeUtils.setCurrentMillisFixed(BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis()); WatermarkEstimator watermarkEstimator = - new WatermarkEstimators.WallTime(new Instant()); + new WatermarkEstimators.WallTime(Instant.now()); DateTimeUtils.setCurrentMillisFixed( BoundedWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(1)).getMillis()); assertEquals( diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterAllTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterAllTest.java index 56e5d369dd61..d8d7e5105af8 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterAllTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterAllTest.java @@ -31,7 +31,7 @@ public class AfterAllTest { @Test public void testFireDeadline() throws Exception { - BoundedWindow window = new IntervalWindow(new Instant(0), new Instant(10)); + BoundedWindow window = new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10)); assertEquals( BoundedWindow.TIMESTAMP_MAX_VALUE, diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterEachTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterEachTest.java index 8ea2fa91432c..680ae6408c0b 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterEachTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterEachTest.java @@ -31,10 +31,10 @@ public class AfterEachTest { @Test public void testFireDeadline() throws Exception { - BoundedWindow window = new IntervalWindow(new Instant(0), new Instant(10)); + BoundedWindow window = new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10)); assertEquals( - new Instant(9), + Instant.ofEpochMilli(9), AfterEach.inOrder(AfterWatermark.pastEndOfWindow(), AfterPane.elementCountAtLeast(4)) .getWatermarkThatGuaranteesFiring(window)); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterFirstTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterFirstTest.java index 0ba77e557015..384128ca00ad 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterFirstTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterFirstTest.java @@ -31,10 +31,10 @@ public class AfterFirstTest { @Test public void testFireDeadline() throws Exception { - BoundedWindow window = new IntervalWindow(new Instant(0), new Instant(10)); + BoundedWindow window = new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10)); assertEquals( - new Instant(9), + Instant.ofEpochMilli(9), AfterFirst.of(AfterWatermark.pastEndOfWindow(), AfterPane.elementCountAtLeast(4)) .getWatermarkThatGuaranteesFiring(window)); assertEquals( diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterPaneTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterPaneTest.java index a352fda8b6b3..d1d36b61267f 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterPaneTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterPaneTest.java @@ -33,7 +33,8 @@ public void testFireDeadline() throws Exception { assertEquals( BoundedWindow.TIMESTAMP_MAX_VALUE, AfterPane.elementCountAtLeast(1) - .getWatermarkThatGuaranteesFiring(new IntervalWindow(new Instant(0), new Instant(10)))); + .getWatermarkThatGuaranteesFiring( + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10)))); } @Test diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTimeTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTimeTest.java index 46f256424d7d..89dd805c9baf 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTimeTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTimeTest.java @@ -36,7 +36,8 @@ public void testFireDeadline() throws Exception { assertEquals( BoundedWindow.TIMESTAMP_MAX_VALUE, AfterProcessingTime.pastFirstElementInPane() - .getWatermarkThatGuaranteesFiring(new IntervalWindow(new Instant(0), new Instant(10)))); + .getWatermarkThatGuaranteesFiring( + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10)))); } @Test diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTimeTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTimeTest.java index 1af271b7ef3f..04134023e825 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTimeTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTimeTest.java @@ -35,7 +35,7 @@ public void testFireDeadline() throws Exception { assertEquals( BoundedWindow.TIMESTAMP_MAX_VALUE, underTest.getWatermarkThatGuaranteesFiring( - new IntervalWindow(new Instant(0), new Instant(10)))); + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10)))); } @Test diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/CalendarWindowsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/CalendarWindowsTest.java index c6af62eacc9b..a66fb9666156 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/CalendarWindowsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/CalendarWindowsTest.java @@ -260,10 +260,10 @@ public void testDefaultWindowMappingFn() { new BoundedWindow() { @Override public Instant maxTimestamp() { - return new Instant(100L); + return Instant.ofEpochMilli(100L); } }), - equalTo(windowFn.assignWindow(new Instant(100L)))); + equalTo(windowFn.assignWindow(Instant.ofEpochMilli(100L)))); assertThat(mapping.maximumLookback(), equalTo(Duration.ZERO)); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/DefaultTriggerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/DefaultTriggerTest.java index d99fd624b418..24768519cec6 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/DefaultTriggerTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/DefaultTriggerTest.java @@ -34,9 +34,10 @@ public class DefaultTriggerTest { @Test public void testFireDeadline() throws Exception { assertEquals( - new Instant(9), + Instant.ofEpochMilli(9), DefaultTrigger.of() - .getWatermarkThatGuaranteesFiring(new IntervalWindow(new Instant(0), new Instant(10)))); + .getWatermarkThatGuaranteesFiring( + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10)))); assertEquals( GlobalWindow.INSTANCE.maxTimestamp(), DefaultTrigger.of().getWatermarkThatGuaranteesFiring(GlobalWindow.INSTANCE)); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/FixedWindowsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/FixedWindowsTest.java index c009e05023ac..ff55bbaf1cba 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/FixedWindowsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/FixedWindowsTest.java @@ -49,9 +49,12 @@ public class FixedWindowsTest { @Test public void testSimpleFixedWindow() throws Exception { Map> expected = new HashMap<>(); - expected.put(new IntervalWindow(new Instant(0), new Instant(10)), set(1, 2, 5, 9)); - expected.put(new IntervalWindow(new Instant(10), new Instant(20)), set(10, 11)); - expected.put(new IntervalWindow(new Instant(100), new Instant(110)), set(100)); + expected.put( + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10)), set(1, 2, 5, 9)); + expected.put( + new IntervalWindow(Instant.ofEpochMilli(10), Instant.ofEpochMilli(20)), set(10, 11)); + expected.put( + new IntervalWindow(Instant.ofEpochMilli(100), Instant.ofEpochMilli(110)), set(100)); assertEquals( expected, runWindowFn( @@ -61,9 +64,10 @@ public void testSimpleFixedWindow() throws Exception { @Test public void testFixedOffsetWindow() throws Exception { Map> expected = new HashMap<>(); - expected.put(new IntervalWindow(new Instant(-5), new Instant(5)), set(1, 2)); - expected.put(new IntervalWindow(new Instant(5), new Instant(15)), set(5, 9, 10, 11)); - expected.put(new IntervalWindow(new Instant(95), new Instant(105)), set(100)); + expected.put(new IntervalWindow(Instant.ofEpochMilli(-5), Instant.ofEpochMilli(5)), set(1, 2)); + expected.put( + new IntervalWindow(Instant.ofEpochMilli(5), Instant.ofEpochMilli(15)), set(5, 9, 10, 11)); + expected.put(new IntervalWindow(Instant.ofEpochMilli(95), Instant.ofEpochMilli(105)), set(100)); assertEquals( expected, runWindowFn( @@ -74,8 +78,12 @@ public void testFixedOffsetWindow() throws Exception { @Test public void testTimeUnit() throws Exception { Map> expected = new HashMap<>(); - expected.put(new IntervalWindow(new Instant(-5000), new Instant(5000)), set(1, 2, 1000)); - expected.put(new IntervalWindow(new Instant(5000), new Instant(15000)), set(5000, 5001, 10000)); + expected.put( + new IntervalWindow(Instant.ofEpochMilli(-5000), Instant.ofEpochMilli(5000)), + set(1, 2, 1000)); + expected.put( + new IntervalWindow(Instant.ofEpochMilli(5000), Instant.ofEpochMilli(15000)), + set(5000, 5001, 10000)); assertEquals( expected, runWindowFn( @@ -93,12 +101,13 @@ public void testDefaultWindowMappingFn() { new BoundedWindow() { @Override public Instant maxTimestamp() { - return new Instant(100L); + return Instant.ofEpochMilli(100L); } }), equalTo( new IntervalWindow( - new Instant(0L), new Instant(0L).plus(Duration.standardMinutes(20L))))); + Instant.ofEpochMilli(0L), + Instant.ofEpochMilli(0L).plus(Duration.standardMinutes(20L))))); assertThat(mapping.maximumLookback(), equalTo(Duration.ZERO)); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/IntervalWindowTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/IntervalWindowTest.java index 3c1a4b4004d0..a7ad6fb20992 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/IntervalWindowTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/IntervalWindowTest.java @@ -42,11 +42,11 @@ public class IntervalWindowTest { private static final List TEST_VALUES = Lists.newArrayList( - new IntervalWindow(new Instant(0), new Instant(0)), - new IntervalWindow(new Instant(0), new Instant(1000)), - new IntervalWindow(new Instant(-1000), new Instant(735)), - new IntervalWindow(new Instant(350), new Instant(60 * 60 * 1000)), - new IntervalWindow(new Instant(0), new Instant(24 * 60 * 60 * 1000)), + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(0)), + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(1000)), + new IntervalWindow(Instant.ofEpochMilli(-1000), Instant.ofEpochMilli(735)), + new IntervalWindow(Instant.ofEpochMilli(350), Instant.ofEpochMilli(60 * 60 * 1000)), + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(24 * 60 * 60 * 1000)), new IntervalWindow( Instant.parse("2015-04-01T00:00:00Z"), Instant.parse("2015-04-01T11:45:13Z"))); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/NeverTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/NeverTest.java index 70a13f9cae84..41ef8005ebdd 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/NeverTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/NeverTest.java @@ -29,7 +29,7 @@ public class NeverTest { @Test public void testFireDeadline() throws Exception { - IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(10)); + IntervalWindow window = new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10)); assertEquals( BoundedWindow.TIMESTAMP_MAX_VALUE, Never.ever().getWatermarkThatGuaranteesFiring(window)); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTriggerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTriggerTest.java index 2ce26b1a930d..21d3cc2cc7fe 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTriggerTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTriggerTest.java @@ -31,24 +31,24 @@ public class OrFinallyTriggerTest { @Test public void testFireDeadline() throws Exception { - BoundedWindow window = new IntervalWindow(new Instant(0), new Instant(10)); + BoundedWindow window = new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10)); assertEquals( - new Instant(9), + Instant.ofEpochMilli(9), Repeatedly.forever(AfterWatermark.pastEndOfWindow()) .getWatermarkThatGuaranteesFiring(window)); assertEquals( - new Instant(9), + Instant.ofEpochMilli(9), Repeatedly.forever(AfterWatermark.pastEndOfWindow()) .orFinally(AfterPane.elementCountAtLeast(1)) .getWatermarkThatGuaranteesFiring(window)); assertEquals( - new Instant(9), + Instant.ofEpochMilli(9), Repeatedly.forever(AfterPane.elementCountAtLeast(1)) .orFinally(AfterWatermark.pastEndOfWindow()) .getWatermarkThatGuaranteesFiring(window)); assertEquals( - new Instant(9), + Instant.ofEpochMilli(9), AfterPane.elementCountAtLeast(100) .orFinally(AfterWatermark.pastEndOfWindow()) .getWatermarkThatGuaranteesFiring(window)); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/RepeatedlyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/RepeatedlyTest.java index 7749c9bac5b3..599a804bd8a0 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/RepeatedlyTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/RepeatedlyTest.java @@ -45,8 +45,8 @@ public void setUp(WindowFn windowFn) throws Exception { @Test public void testFireDeadline() throws Exception { setUp(FixedWindows.of(Duration.millis(10))); - IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(10)); - Instant arbitraryInstant = new Instant(34957849); + IntervalWindow window = new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10)); + Instant arbitraryInstant = Instant.ofEpochMilli(34957849); when(mockTrigger.getWatermarkThatGuaranteesFiring(Mockito.any())) .thenReturn(arbitraryInstant); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SessionsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SessionsTest.java index 783fc3b37265..55d99a4ced43 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SessionsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SessionsTest.java @@ -50,9 +50,10 @@ public class SessionsTest { @Test public void testSimple() throws Exception { Map> expected = new HashMap<>(); - expected.put(new IntervalWindow(new Instant(0), new Instant(10)), set(0)); - expected.put(new IntervalWindow(new Instant(10), new Instant(20)), set(10)); - expected.put(new IntervalWindow(new Instant(101), new Instant(111)), set(101)); + expected.put(new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10)), set(0)); + expected.put(new IntervalWindow(Instant.ofEpochMilli(10), Instant.ofEpochMilli(20)), set(10)); + expected.put( + new IntervalWindow(Instant.ofEpochMilli(101), Instant.ofEpochMilli(111)), set(101)); assertEquals( expected, runWindowFn(Sessions.withGapDuration(Duration.millis(10)), Arrays.asList(0L, 10L, 101L))); @@ -61,8 +62,10 @@ public void testSimple() throws Exception { @Test public void testConsecutive() throws Exception { Map> expected = new HashMap<>(); - expected.put(new IntervalWindow(new Instant(1), new Instant(19)), set(1, 2, 5, 9)); - expected.put(new IntervalWindow(new Instant(100), new Instant(111)), set(100, 101)); + expected.put( + new IntervalWindow(Instant.ofEpochMilli(1), Instant.ofEpochMilli(19)), set(1, 2, 5, 9)); + expected.put( + new IntervalWindow(Instant.ofEpochMilli(100), Instant.ofEpochMilli(111)), set(100, 101)); assertEquals( expected, runWindowFn( @@ -73,8 +76,11 @@ public void testConsecutive() throws Exception { @Test public void testMerging() throws Exception { Map> expected = new HashMap<>(); - expected.put(new IntervalWindow(new Instant(1), new Instant(40)), set(1, 10, 15, 22, 30)); - expected.put(new IntervalWindow(new Instant(95), new Instant(111)), set(95, 100, 101)); + expected.put( + new IntervalWindow(Instant.ofEpochMilli(1), Instant.ofEpochMilli(40)), + set(1, 10, 15, 22, 30)); + expected.put( + new IntervalWindow(Instant.ofEpochMilli(95), Instant.ofEpochMilli(111)), set(95, 100, 101)); assertEquals( expected, runWindowFn( @@ -85,9 +91,13 @@ public void testMerging() throws Exception { @Test public void testTimeUnit() throws Exception { Map> expected = new HashMap<>(); - expected.put(new IntervalWindow(new Instant(1), new Instant(2000)), set(1, 2, 1000)); - expected.put(new IntervalWindow(new Instant(5000), new Instant(6001)), set(5000, 5001)); - expected.put(new IntervalWindow(new Instant(10000), new Instant(11000)), set(10000)); + expected.put( + new IntervalWindow(Instant.ofEpochMilli(1), Instant.ofEpochMilli(2000)), set(1, 2, 1000)); + expected.put( + new IntervalWindow(Instant.ofEpochMilli(5000), Instant.ofEpochMilli(6001)), + set(5000, 5001)); + expected.put( + new IntervalWindow(Instant.ofEpochMilli(10000), Instant.ofEpochMilli(11000)), set(10000)); assertEquals( expected, runWindowFn( diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SlidingWindowsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SlidingWindowsTest.java index df463e838073..7fa91c8fdca6 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SlidingWindowsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SlidingWindowsTest.java @@ -48,10 +48,13 @@ public class SlidingWindowsTest { @Test public void testSimple() throws Exception { Map> expected = new HashMap<>(); - expected.put(new IntervalWindow(new Instant(-5), new Instant(5)), set(1, 2)); - expected.put(new IntervalWindow(new Instant(0), new Instant(10)), set(1, 2, 5, 9)); - expected.put(new IntervalWindow(new Instant(5), new Instant(15)), set(5, 9, 10, 11)); - expected.put(new IntervalWindow(new Instant(10), new Instant(20)), set(10, 11)); + expected.put(new IntervalWindow(Instant.ofEpochMilli(-5), Instant.ofEpochMilli(5)), set(1, 2)); + expected.put( + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10)), set(1, 2, 5, 9)); + expected.put( + new IntervalWindow(Instant.ofEpochMilli(5), Instant.ofEpochMilli(15)), set(5, 9, 10, 11)); + expected.put( + new IntervalWindow(Instant.ofEpochMilli(10), Instant.ofEpochMilli(20)), set(10, 11)); SlidingWindows windowFn = SlidingWindows.of(Duration.millis(10)).every(Duration.millis(5)); assertEquals(expected, runWindowFn(windowFn, Arrays.asList(1L, 2L, 5L, 9L, 10L, 11L))); assertThat(windowFn.assignsToOneWindow(), is(false)); @@ -60,10 +63,13 @@ public void testSimple() throws Exception { @Test public void testSlightlyOverlapping() throws Exception { Map> expected = new HashMap<>(); - expected.put(new IntervalWindow(new Instant(-5), new Instant(2)), set(1)); - expected.put(new IntervalWindow(new Instant(0), new Instant(7)), set(1, 2, 5)); - expected.put(new IntervalWindow(new Instant(5), new Instant(12)), set(5, 9, 10, 11)); - expected.put(new IntervalWindow(new Instant(10), new Instant(17)), set(10, 11)); + expected.put(new IntervalWindow(Instant.ofEpochMilli(-5), Instant.ofEpochMilli(2)), set(1)); + expected.put( + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(7)), set(1, 2, 5)); + expected.put( + new IntervalWindow(Instant.ofEpochMilli(5), Instant.ofEpochMilli(12)), set(5, 9, 10, 11)); + expected.put( + new IntervalWindow(Instant.ofEpochMilli(10), Instant.ofEpochMilli(17)), set(10, 11)); SlidingWindows windowFn = SlidingWindows.of(Duration.millis(7)).every(Duration.millis(5)); assertEquals(expected, runWindowFn(windowFn, Arrays.asList(1L, 2L, 5L, 9L, 10L, 11L))); assertThat(windowFn.assignsToOneWindow(), is(false)); @@ -72,9 +78,10 @@ public void testSlightlyOverlapping() throws Exception { @Test public void testEqualSize() throws Exception { Map> expected = new HashMap<>(); - expected.put(new IntervalWindow(new Instant(0), new Instant(3)), set(1, 2)); - expected.put(new IntervalWindow(new Instant(3), new Instant(6)), set(3, 4, 5)); - expected.put(new IntervalWindow(new Instant(6), new Instant(9)), set(6, 7)); + expected.put(new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(3)), set(1, 2)); + expected.put( + new IntervalWindow(Instant.ofEpochMilli(3), Instant.ofEpochMilli(6)), set(3, 4, 5)); + expected.put(new IntervalWindow(Instant.ofEpochMilli(6), Instant.ofEpochMilli(9)), set(6, 7)); SlidingWindows windowFn = SlidingWindows.of(Duration.millis(3)).every(Duration.millis(3)); assertEquals(expected, runWindowFn(windowFn, Arrays.asList(1L, 2L, 3L, 4L, 5L, 6L, 7L))); assertThat(windowFn.assignsToOneWindow(), is(true)); @@ -83,9 +90,11 @@ public void testEqualSize() throws Exception { @Test public void testElidings() throws Exception { Map> expected = new HashMap<>(); - expected.put(new IntervalWindow(new Instant(0), new Instant(3)), set(1, 2)); - expected.put(new IntervalWindow(new Instant(10), new Instant(13)), set(10, 11)); - expected.put(new IntervalWindow(new Instant(100), new Instant(103)), set(100)); + expected.put(new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(3)), set(1, 2)); + expected.put( + new IntervalWindow(Instant.ofEpochMilli(10), Instant.ofEpochMilli(13)), set(10, 11)); + expected.put( + new IntervalWindow(Instant.ofEpochMilli(100), Instant.ofEpochMilli(103)), set(100)); SlidingWindows windowFn = SlidingWindows.of(Duration.millis(3)).every(Duration.millis(10)); assertEquals( expected, @@ -98,10 +107,14 @@ public void testElidings() throws Exception { @Test public void testOffset() throws Exception { Map> expected = new HashMap<>(); - expected.put(new IntervalWindow(new Instant(-8), new Instant(2)), set(1)); - expected.put(new IntervalWindow(new Instant(-3), new Instant(7)), set(1, 2, 5)); - expected.put(new IntervalWindow(new Instant(2), new Instant(12)), set(2, 5, 9, 10, 11)); - expected.put(new IntervalWindow(new Instant(7), new Instant(17)), set(9, 10, 11)); + expected.put(new IntervalWindow(Instant.ofEpochMilli(-8), Instant.ofEpochMilli(2)), set(1)); + expected.put( + new IntervalWindow(Instant.ofEpochMilli(-3), Instant.ofEpochMilli(7)), set(1, 2, 5)); + expected.put( + new IntervalWindow(Instant.ofEpochMilli(2), Instant.ofEpochMilli(12)), + set(2, 5, 9, 10, 11)); + expected.put( + new IntervalWindow(Instant.ofEpochMilli(7), Instant.ofEpochMilli(17)), set(9, 10, 11)); assertEquals( expected, runWindowFn( @@ -114,11 +127,17 @@ public void testOffset() throws Exception { @Test public void testTimeUnit() throws Exception { Map> expected = new HashMap<>(); - expected.put(new IntervalWindow(new Instant(-5000), new Instant(5000)), set(1, 2, 1000)); expected.put( - new IntervalWindow(new Instant(0), new Instant(10000)), set(1, 2, 1000, 5000, 5001)); - expected.put(new IntervalWindow(new Instant(5000), new Instant(15000)), set(5000, 5001, 10000)); - expected.put(new IntervalWindow(new Instant(10000), new Instant(20000)), set(10000)); + new IntervalWindow(Instant.ofEpochMilli(-5000), Instant.ofEpochMilli(5000)), + set(1, 2, 1000)); + expected.put( + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(10000)), + set(1, 2, 1000, 5000, 5001)); + expected.put( + new IntervalWindow(Instant.ofEpochMilli(5000), Instant.ofEpochMilli(15000)), + set(5000, 5001, 10000)); + expected.put( + new IntervalWindow(Instant.ofEpochMilli(10000), Instant.ofEpochMilli(20000)), set(10000)); assertEquals( expected, runWindowFn( @@ -184,19 +203,23 @@ public void testDefaultWindowMappingFn() { // Prior assertEquals( - new IntervalWindow(new Instant(340), new Instant(1340)), - mapping.getSideInputWindow(new IntervalWindow(new Instant(0), new Instant(1041)))); + new IntervalWindow(Instant.ofEpochMilli(340), Instant.ofEpochMilli(1340)), + mapping.getSideInputWindow( + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(1041)))); assertEquals( - new IntervalWindow(new Instant(340), new Instant(1340)), - mapping.getSideInputWindow(new IntervalWindow(new Instant(0), new Instant(1339)))); + new IntervalWindow(Instant.ofEpochMilli(340), Instant.ofEpochMilli(1340)), + mapping.getSideInputWindow( + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(1339)))); // Align assertEquals( - new IntervalWindow(new Instant(340), new Instant(1340)), - mapping.getSideInputWindow(new IntervalWindow(new Instant(0), new Instant(1340)))); + new IntervalWindow(Instant.ofEpochMilli(340), Instant.ofEpochMilli(1340)), + mapping.getSideInputWindow( + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(1340)))); // After assertEquals( - new IntervalWindow(new Instant(640), new Instant(1640)), - mapping.getSideInputWindow(new IntervalWindow(new Instant(0), new Instant(1341)))); + new IntervalWindow(Instant.ofEpochMilli(640), Instant.ofEpochMilli(1640)), + mapping.getSideInputWindow( + new IntervalWindow(Instant.ofEpochMilli(0), Instant.ofEpochMilli(1341)))); } @Test diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java index 81cb7ffc02ba..4a45c1104021 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java @@ -450,8 +450,8 @@ public void testTimestampCombinerDefault() { pipeline .apply( Create.timestamped( - TimestampedValue.of(KV.of(0, "hello"), new Instant(0)), - TimestampedValue.of(KV.of(0, "goodbye"), new Instant(10)))) + TimestampedValue.of(KV.of(0, "hello"), Instant.ofEpochMilli(0)), + TimestampedValue.of(KV.of(0, "goodbye"), Instant.ofEpochMilli(10)))) .apply(Window.into(FixedWindows.of(Duration.standardMinutes(10)))) .apply(GroupByKey.create()) .apply( @@ -463,8 +463,8 @@ public void processElement(ProcessContext c) throws Exception { c.timestamp(), equalTo( new IntervalWindow( - new Instant(0), - new Instant(0).plus(Duration.standardMinutes(10))) + Instant.ofEpochMilli(0), + Instant.ofEpochMilli(0).plus(Duration.standardMinutes(10))) .maxTimestamp())); } })); @@ -484,8 +484,8 @@ public void testTimestampCombinerEndOfWindow() { pipeline .apply( Create.timestamped( - TimestampedValue.of(KV.of(0, "hello"), new Instant(0)), - TimestampedValue.of(KV.of(0, "goodbye"), new Instant(10)))) + TimestampedValue.of(KV.of(0, "hello"), Instant.ofEpochMilli(0)), + TimestampedValue.of(KV.of(0, "goodbye"), Instant.ofEpochMilli(10)))) .apply( Window.>into(FixedWindows.of(Duration.standardMinutes(10))) .withTimestampCombiner(TimestampCombiner.END_OF_WINDOW)) @@ -495,7 +495,7 @@ public void testTimestampCombinerEndOfWindow() { new DoFn>, Void>() { @ProcessElement public void processElement(ProcessContext c) throws Exception { - assertThat(c.timestamp(), equalTo(new Instant(10 * 60 * 1000 - 1))); + assertThat(c.timestamp(), equalTo(Instant.ofEpochMilli(10 * 60 * 1000 - 1))); } })); @@ -618,7 +618,7 @@ public void testDisplayDataExcludesDefaults() { @Test @Category({ValidatesRunner.class, UsesCustomWindowMerging.class}) public void testMergingCustomWindows() { - Instant startInstant = new Instant(0L); + Instant startInstant = Instant.ofEpochMilli(0L); PCollection inputCollection = pipeline.apply( Create.timestamped( @@ -641,7 +641,7 @@ public void testMergingCustomWindows() { @Test @Category({ValidatesRunner.class, UsesCustomWindowMerging.class}) public void testMergingCustomWindowsKeyedCollection() { - Instant startInstant = new Instant(0L); + Instant startInstant = Instant.ofEpochMilli(0L); PCollection> inputCollection = pipeline.apply( Create.timestamped( @@ -666,7 +666,7 @@ public void testMergingCustomWindowsKeyedCollection() { @Test @Category({ValidatesRunner.class, UsesCustomWindowMerging.class}) public void testMergingCustomWindowsWithoutCustomWindowTypes() { - Instant startInstant = new Instant(0L); + Instant startInstant = Instant.ofEpochMilli(0L); PCollection> inputCollection = pipeline.apply( Create.timestamped( diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowingTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowingTest.java index c1c3e86f5711..ca99e2a9e85b 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowingTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowingTest.java @@ -96,9 +96,9 @@ private String output(String value, int count, int timestamp, int windowStart, i + ":" + timestamp + ":[" - + new Instant(windowStart) + + Instant.ofEpochMilli(windowStart) + ".." - + new Instant(windowEnd) + + Instant.ofEpochMilli(windowEnd) + ")"; } @@ -108,11 +108,11 @@ public void testPartitioningWindowing() { PCollection input = p.apply( Create.timestamped( - TimestampedValue.of("a", new Instant(1)), - TimestampedValue.of("b", new Instant(2)), - TimestampedValue.of("b", new Instant(3)), - TimestampedValue.of("c", new Instant(11)), - TimestampedValue.of("d", new Instant(11)))); + TimestampedValue.of("a", Instant.ofEpochMilli(1)), + TimestampedValue.of("b", Instant.ofEpochMilli(2)), + TimestampedValue.of("b", Instant.ofEpochMilli(3)), + TimestampedValue.of("c", Instant.ofEpochMilli(11)), + TimestampedValue.of("d", Instant.ofEpochMilli(11)))); PCollection output = input.apply(new WindowedCount(FixedWindows.of(Duration.millis(10)))); @@ -133,9 +133,9 @@ public void testNonPartitioningWindowing() { PCollection input = p.apply( Create.timestamped( - TimestampedValue.of("a", new Instant(1)), - TimestampedValue.of("a", new Instant(7)), - TimestampedValue.of("b", new Instant(8)))); + TimestampedValue.of("a", Instant.ofEpochMilli(1)), + TimestampedValue.of("a", Instant.ofEpochMilli(7)), + TimestampedValue.of("b", Instant.ofEpochMilli(8)))); PCollection output = input.apply( @@ -158,9 +158,9 @@ public void testMergingWindowing() { PCollection input = p.apply( Create.timestamped( - TimestampedValue.of("a", new Instant(1)), - TimestampedValue.of("a", new Instant(5)), - TimestampedValue.of("a", new Instant(20)))); + TimestampedValue.of("a", Instant.ofEpochMilli(1)), + TimestampedValue.of("a", Instant.ofEpochMilli(5)), + TimestampedValue.of("a", Instant.ofEpochMilli(20)))); PCollection output = input.apply(new WindowedCount(Sessions.withGapDuration(Duration.millis(10)))); @@ -177,15 +177,15 @@ public void testWindowPreservation() { p.apply( "Create12", Create.timestamped( - TimestampedValue.of("a", new Instant(1)), - TimestampedValue.of("b", new Instant(2)))); + TimestampedValue.of("a", Instant.ofEpochMilli(1)), + TimestampedValue.of("b", Instant.ofEpochMilli(2)))); PCollection input2 = p.apply( "Create34", Create.timestamped( - TimestampedValue.of("a", new Instant(3)), - TimestampedValue.of("b", new Instant(4)))); + TimestampedValue.of("a", Instant.ofEpochMilli(3)), + TimestampedValue.of("b", Instant.ofEpochMilli(4)))); PCollectionList input = PCollectionList.of(input1).and(input2); @@ -248,7 +248,7 @@ static class ExtractWordsWithTimestampsFn extends DoFn { public void processElement(ProcessContext c) { List words = Splitter.onPattern("[^a-zA-Z0-9']+").splitToList(c.element()); if (words.size() == 2) { - c.outputWithTimestamp(words.get(0), new Instant(Long.parseLong(words.get(1)))); + c.outputWithTimestamp(words.get(0), Instant.ofEpochMilli(Long.parseLong(words.get(1)))); } } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/RowStringInterpolatorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/RowStringInterpolatorTest.java index fed55cc4cb89..dbb633aa3921 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/RowStringInterpolatorTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/RowStringInterpolatorTest.java @@ -71,7 +71,7 @@ public void testInvalidRowThrowsHelpfulError() { interpolator.interpolate( ValueInSingleWindow.of( - invalidRow, new Instant(0), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); + invalidRow, Instant.ofEpochMilli(0), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); } @Test @@ -90,7 +90,7 @@ public void testInvalidRowThrowsHelpfulErrorForNestedFields() { interpolator.interpolate( ValueInSingleWindow.of( - invalidRow, new Instant(0), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); + invalidRow, Instant.ofEpochMilli(0), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); } @Test @@ -112,7 +112,7 @@ public void testInvalidRowThrowsHelpfulErrorForDoublyNestedFields() { interpolator.interpolate( ValueInSingleWindow.of( - invalidRow, new Instant(0), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); + invalidRow, Instant.ofEpochMilli(0), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); } private static final Row ROW = @@ -143,7 +143,8 @@ public void testTopLevelInterpolation() { String output = interpolator.interpolate( - ValueInSingleWindow.of(ROW, new Instant(0), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); + ValueInSingleWindow.of( + ROW, Instant.ofEpochMilli(0), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); assertEquals("foo str_value, bar true, baz 123, xyz ", output); } @@ -155,7 +156,8 @@ public void testNestedLevelInterpolation() { String output = interpolator.interpolate( - ValueInSingleWindow.of(ROW, new Instant(0), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); + ValueInSingleWindow.of( + ROW, Instant.ofEpochMilli(0), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); assertEquals("foo str_value, bar nested_str_value, baz 1.234", output); } @@ -168,7 +170,8 @@ public void testDoublyNestedInterpolation() { String output = interpolator.interpolate( - ValueInSingleWindow.of(ROW, new Instant(0), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); + ValueInSingleWindow.of( + ROW, Instant.ofEpochMilli(0), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); assertEquals("foo str_value, bar doubly_nested_str_value, baz 789", output); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/WindowedValueTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/WindowedValueTest.java index 97b99321e5c7..055b5f7082b3 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/WindowedValueTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/WindowedValueTest.java @@ -55,11 +55,11 @@ public class WindowedValueTest { @Test public void testWindowedValueCoder() throws CoderException { - Instant timestamp = new Instant(1234); + Instant timestamp = Instant.ofEpochMilli(1234); WindowedValue value = WindowedValues.of( "abc", - new Instant(1234), + Instant.ofEpochMilli(1234), Arrays.asList( new IntervalWindow(timestamp, timestamp.plus(Duration.millis(1000))), new IntervalWindow( @@ -81,11 +81,11 @@ public void testWindowedValueCoder() throws CoderException { @Test public void testWindowedValueWithElementMetadataCoder() throws CoderException { WindowedValues.WindowedValueCoder.setMetadataSupported(); - Instant timestamp = new Instant(1234); + Instant timestamp = Instant.ofEpochMilli(1234); WindowedValue value = WindowedValues.of( "abc", - new Instant(1234), + Instant.ofEpochMilli(1234), Arrays.asList( new IntervalWindow(timestamp, timestamp.plus(Duration.millis(1000))), new IntervalWindow( diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/PCollectionTranslationTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/PCollectionTranslationTest.java index 1b7451f53ace..4809d3f562e0 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/PCollectionTranslationTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/PCollectionTranslationTest.java @@ -175,7 +175,7 @@ public Collection assignWindows(final AssignContext c) throws Exc new BoundedWindow() { @Override public Instant maxTimestamp() { - return new Instant(c.element().longValue()); + return Instant.ofEpochMilli(c.element().longValue()); } }); } @@ -209,7 +209,7 @@ public void encode(BoundedWindow value, OutputStream outStream) throws IOExcepti @Override public BoundedWindow decode(InputStream inStream) throws IOException { - final Instant ts = new Instant(VarInt.decodeLong(inStream)); + final Instant ts = Instant.ofEpochMilli(VarInt.decodeLong(inStream)); return new BoundedWindow() { @Override public Instant maxTimestamp() { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/TestStreamTranslationTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/TestStreamTranslationTest.java index 068a9781a960..fd9e5d4c21fc 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/TestStreamTranslationTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/TestStreamTranslationTest.java @@ -50,10 +50,10 @@ public static Iterable> data() { return ImmutableList.of( TestStream.create(VarIntCoder.of()).advanceWatermarkToInfinity(), TestStream.create(VarIntCoder.of()) - .advanceWatermarkTo(new Instant(42)) + .advanceWatermarkTo(Instant.ofEpochMilli(42)) .advanceWatermarkToInfinity(), TestStream.create(VarIntCoder.of()) - .addElements(TimestampedValue.of(3, new Instant(17))) + .addElements(TimestampedValue.of(3, Instant.ofEpochMilli(17))) .advanceWatermarkToInfinity(), TestStream.create(StringUtf8Coder.of()) .advanceProcessingTime(Duration.millis(82)) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/TimerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/TimerTest.java index 06f43350f5ae..d0a98787c773 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/TimerTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/TimerTest.java @@ -36,8 +36,8 @@ /** Tests for {@link Timer}. */ @RunWith(JUnit4.class) public class TimerTest { - private static final Instant FIRE_TIME = new Instant(123L); - private static final Instant HOLD_TIME = new Instant(456L); + private static final Instant FIRE_TIME = Instant.ofEpochMilli(123L); + private static final Instant HOLD_TIME = Instant.ofEpochMilli(456L); @Test public void testClearTimer() { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/TriggerTranslationTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/TriggerTranslationTest.java index 1ad33e18e2f0..1152149b30b1 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/TriggerTranslationTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/TriggerTranslationTest.java @@ -68,11 +68,11 @@ public static Iterable data() { AfterProcessingTime.pastFirstElementInPane().plusDelayOf(Duration.millis(23))), toProtoAndBackSpec( AfterProcessingTime.pastFirstElementInPane() - .alignedTo(Duration.millis(5), new Instant(27))), + .alignedTo(Duration.millis(5), Instant.ofEpochMilli(27))), toProtoAndBackSpec( AfterProcessingTime.pastFirstElementInPane() .plusDelayOf(Duration.standardSeconds(3)) - .alignedTo(Duration.millis(5), new Instant(27)) + .alignedTo(Duration.millis(5), Instant.ofEpochMilli(27)) .plusDelayOf(Duration.millis(13))), // Composite triggers diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/UnboundedReadFromBoundedSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/UnboundedReadFromBoundedSourceTest.java index 3279f66d5c6f..2ccc4ea532a4 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/UnboundedReadFromBoundedSourceTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/UnboundedReadFromBoundedSourceTest.java @@ -284,7 +284,7 @@ public void testReadFromCheckpointBeforeStart() throws Exception { PipelineOptions options = PipelineOptionsFactory.create(); List> elements = - ImmutableList.of(TimestampedValue.of(1L, new Instant(1L))); + ImmutableList.of(TimestampedValue.of(1L, Instant.ofEpochMilli(1L))); Checkpoint checkpoint = new Checkpoint<>(elements, countingSource); unboundedSource.createReader(options, checkpoint).getCurrent(); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/renderer/PipelineDotRendererTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/renderer/PipelineDotRendererTest.java index 4e4fc291630a..4603ec456059 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/renderer/PipelineDotRendererTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/renderer/PipelineDotRendererTest.java @@ -54,8 +54,8 @@ public void testEmptyPipeline() { public void testCompositePipeline() { p.apply( Create.timestamped( - TimestampedValue.of(KV.of(1, 1), new Instant(1)), - TimestampedValue.of(KV.of(2, 2), new Instant(2)))) + TimestampedValue.of(KV.of(1, 1), Instant.ofEpochMilli(1)), + TimestampedValue.of(KV.of(2, 2), Instant.ofEpochMilli(2)))) .apply(Window.into(FixedWindows.of(Duration.millis(10)))) .apply(Sum.integersPerKey()); assertEquals( diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/renderer/PortablePipelineDotRendererTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/renderer/PortablePipelineDotRendererTest.java index e957ecfcd4c4..0918a232d75b 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/renderer/PortablePipelineDotRendererTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/renderer/PortablePipelineDotRendererTest.java @@ -49,7 +49,7 @@ public void testEmptyPipeline() { @Test public void testCompositePipeline() { - p.apply(Create.timestamped(TimestampedValue.of(KV.of(1, 1), new Instant(1)))) + p.apply(Create.timestamped(TimestampedValue.of(KV.of(1, 1), Instant.ofEpochMilli(1)))) .apply(Window.into(FixedWindows.of(Duration.millis(10)))) .apply(Sum.integersPerKey()); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/RowTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/RowTest.java index 37583f0ceedc..5e7577d518d1 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/RowTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/RowTest.java @@ -128,7 +128,7 @@ public void testCreatesRecord() { .build(); DateTime dateTime = - new DateTime().withDate(1979, 03, 14).withTime(1, 2, 3, 4).withZone(DateTimeZone.UTC); + DateTime.now().withDate(1979, 03, 14).withTime(1, 2, 3, 4).withZone(DateTimeZone.UTC); Row row = Row.withSchema(schema) .addValues( @@ -572,7 +572,7 @@ public void testCreateWithNames() { .collect(toSchema()); DateTime dateTime = - new DateTime().withDate(1979, 03, 14).withTime(1, 2, 3, 4).withZone(DateTimeZone.UTC); + DateTime.now().withDate(1979, 03, 14).withTime(1, 2, 3, 4).withZone(DateTimeZone.UTC); byte[] bytes = new byte[] {1, 2, 3, 4}; Row row = diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TimestampedValueTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TimestampedValueTest.java index 2a246fefbb7a..e9696b070d43 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TimestampedValueTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TimestampedValueTest.java @@ -68,10 +68,10 @@ public void testNullValue() { public void testEquality() { new EqualsTester() .addEqualityGroup( - TimestampedValue.of("foo", new Instant(1000)), - TimestampedValue.of("foo", new Instant(1000))) - .addEqualityGroup(TimestampedValue.of("foo", new Instant(2000))) - .addEqualityGroup(TimestampedValue.of("bar", new Instant(1000))) + TimestampedValue.of("foo", Instant.ofEpochMilli(1000)), + TimestampedValue.of("foo", Instant.ofEpochMilli(1000))) + .addEqualityGroup(TimestampedValue.of("foo", Instant.ofEpochMilli(2000))) + .addEqualityGroup(TimestampedValue.of("bar", Instant.ofEpochMilli(1000))) .addEqualityGroup( TimestampedValue.of("foo", BoundedWindow.TIMESTAMP_MIN_VALUE), TimestampedValue.atMinimumTimestamp("foo")) diff --git a/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/schemas/utils/AvroUtils.java b/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/schemas/utils/AvroUtils.java index 882e46208a96..70311aa3042d 100644 --- a/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/schemas/utils/AvroUtils.java +++ b/sdks/java/extensions/avro/src/main/java/org/apache/beam/sdk/extensions/avro/schemas/utils/AvroUtils.java @@ -1632,7 +1632,7 @@ private static Object convertDateStrict(Integer epochDays, FieldType fieldType) private static Object convertDateTimeStrict(Long value, FieldType fieldType) { checkTypeName(fieldType.getTypeName(), TypeName.DATETIME, "dateTime"); - return new Instant(value); + return Instant.ofEpochMilli(value); } private static Object convertFloatStrict(Float value, FieldType fieldType) { diff --git a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/coders/AvroCoderTest.java b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/coders/AvroCoderTest.java index 2c0337979c05..0adcc485648b 100644 --- a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/coders/AvroCoderTest.java +++ b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/coders/AvroCoderTest.java @@ -110,9 +110,9 @@ public class AvroCoderTest { public static final DateTime DATETIME_A = - new DateTime().withDate(1994, 10, 31).withZone(DateTimeZone.UTC); + DateTime.now().withDate(1994, 10, 31).withZone(DateTimeZone.UTC); public static final DateTime DATETIME_B = - new DateTime().withDate(1997, 4, 25).withZone(DateTimeZone.UTC); + DateTime.now().withDate(1997, 4, 25).withZone(DateTimeZone.UTC); private static final TestAvroNested AVRO_NESTED_SPECIFIC_RECORD = new TestAvroNested(true, 42); private static final TestAvro AVRO_SPECIFIC_RECORD = TestAvroFactory.newInstance( @@ -125,7 +125,7 @@ public class AvroCoderTest { ByteBuffer.wrap(new byte[] {1, 2, 3, 4}), new fixed4(new byte[] {1, 2, 3, 4}), new LocalDate(1979, 3, 14), - new DateTime().withDate(1979, 3, 14).withTime(1, 2, 3, 4), + DateTime.now().withDate(1979, 3, 14).withTime(1, 2, 3, 4), TestEnum.abc, AVRO_NESTED_SPECIFIC_RECORD, ImmutableList.of(AVRO_NESTED_SPECIFIC_RECORD, AVRO_NESTED_SPECIFIC_RECORD), diff --git a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/io/AvroIOTest.java b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/io/AvroIOTest.java index 2a0bc36f6e9e..2dd0089953b9 100644 --- a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/io/AvroIOTest.java +++ b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/io/AvroIOTest.java @@ -913,7 +913,7 @@ void testWindowedAvroIOWriteUsingMethod(WriteMethod method) throws IOException { Path baseDir = Files.createTempDirectory(tmpFolder.getRoot().toPath(), "testwrite"); final String baseFilename = baseDir.resolve("prefix").toString(); - Instant base = new Instant(0); + Instant base = Instant.ofEpochMilli(0); ArrayList allElements = new ArrayList<>(); ArrayList> firstWindowElements = new ArrayList<>(); ArrayList firstWindowTimestamps = @@ -950,11 +950,11 @@ void testWindowedAvroIOWriteUsingMethod(WriteMethod method) throws IOException { TestStream values = TestStream.create(AvroCoder.of(GenericClass.class)) - .advanceWatermarkTo(new Instant(0)) + .advanceWatermarkTo(Instant.ofEpochMilli(0)) .addElements( firstWindowArray[0], Arrays.copyOfRange(firstWindowArray, 1, firstWindowArray.length)) - .advanceWatermarkTo(new Instant(0).plus(Duration.standardMinutes(1))) + .advanceWatermarkTo(Instant.ofEpochMilli(0).plus(Duration.standardMinutes(1))) .addElements( secondWindowArray[0], Arrays.copyOfRange(secondWindowArray, 1, secondWindowArray.length)) @@ -1005,7 +1005,7 @@ void testWindowedAvroIOWriteUsingMethod(WriteMethod method) throws IOException { List expectedFiles = new ArrayList<>(); for (int shard = 0; shard < 2; shard++) { for (int window = 0; window < 2; window++) { - Instant windowStart = new Instant(0).plus(Duration.standardMinutes(window)); + Instant windowStart = Instant.ofEpochMilli(0).plus(Duration.standardMinutes(window)); IntervalWindow iw = new IntervalWindow(windowStart, Duration.standardMinutes(1)); String baseAndWindow = baseFilename + "-" + iw.start() + "-" + iw.end(); switch (method) { diff --git a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/io/AvroSchemaIOProviderTest.java b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/io/AvroSchemaIOProviderTest.java index b003597200eb..f886eb73bd7e 100644 --- a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/io/AvroSchemaIOProviderTest.java +++ b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/io/AvroSchemaIOProviderTest.java @@ -89,7 +89,7 @@ public void testStreamingWriteDefault() throws Exception { TestStream createEvents = TestStream.create(RowCoder.of(SCHEMA)) - .addElements(TimestampedValue.of(createRow(1L), new Instant(1L))) + .addElements(TimestampedValue.of(createRow(1L), Instant.ofEpochMilli(1L))) .addElements(TimestampedValue.of(createRow(2L), Instant.ofEpochSecond(120L))) .advanceWatermarkToInfinity(); @@ -122,7 +122,7 @@ public void testStreamingCustomWindowSize() throws Exception { TestStream createEvents = TestStream.create(RowCoder.of(SCHEMA)) - .addElements(TimestampedValue.of(createRow(1L), new Instant(1L))) + .addElements(TimestampedValue.of(createRow(1L), Instant.ofEpochMilli(1L))) .addElements(TimestampedValue.of(createRow(2L), Instant.ofEpochSecond(120L))) .advanceWatermarkToInfinity(); diff --git a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/AvroSchemaTest.java b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/AvroSchemaTest.java index 1acf77e41549..5f0d9650081c 100644 --- a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/AvroSchemaTest.java +++ b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/AvroSchemaTest.java @@ -302,7 +302,7 @@ public String toString() { private static final byte[] BYTE_ARRAY = new byte[] {1, 2, 3, 4}; private static final DateTime DATE_TIME = - new DateTime().withDate(1979, 3, 14).withTime(1, 2, 3, 4); + DateTime.now().withDate(1979, 3, 14).withTime(1, 2, 3, 4); private static final LocalDate DATE = new LocalDate(1979, 3, 14); private static final TestAvroNested AVRO_NESTED_SPECIFIC_RECORD = new TestAvroNested(true, 42); private static final TestAvro AVRO_SPECIFIC_RECORD = diff --git a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/SchemaCoderTest.java b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/SchemaCoderTest.java index 0cda00a6e86a..ad8e4767e07c 100644 --- a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/SchemaCoderTest.java +++ b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/SchemaCoderTest.java @@ -129,13 +129,13 @@ public static Collection data() throws NoSuchSchemaException { "foo", 9001, 0L, - new DateTime().withDate(1979, 3, 14).withTime(10, 30, 0, 0)), + DateTime.now().withDate(1979, 3, 14).withTime(10, 30, 0, 0)), () -> new SimpleAvro( "bar", 9002, 1L, - new DateTime().withDate(1989, 3, 14).withTime(10, 30, 0, 0))), + DateTime.now().withDate(1989, 3, 14).withTime(10, 30, 0, 0))), true }, new Object[] { diff --git a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/utils/AvroUtilsTest.java b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/utils/AvroUtilsTest.java index d087ed0a20bc..7d0533a721f4 100644 --- a/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/utils/AvroUtilsTest.java +++ b/sdks/java/extensions/avro/src/test/java/org/apache/beam/sdk/extensions/avro/schemas/utils/AvroUtilsTest.java @@ -457,7 +457,7 @@ private Schema getBeamSchema() { private static final byte[] BYTE_ARRAY = new byte[] {1, 2, 3, 4}; private static final DateTime DATE_TIME = - new DateTime().withDate(1979, 3, 14).withTime(1, 2, 3, 4).withZone(DateTimeZone.UTC); + DateTime.now().withDate(1979, 3, 14).withTime(1, 2, 3, 4).withZone(DateTimeZone.UTC); private static final BigDecimal BIG_DECIMAL = new BigDecimal(3600); private Row getBeamRow() { diff --git a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/FlatMapTranslator.java b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/FlatMapTranslator.java index 62d047a8db43..64f5cf1bcae7 100644 --- a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/FlatMapTranslator.java +++ b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/FlatMapTranslator.java @@ -106,7 +106,8 @@ private Collector(@Nullable ExtractEventTime eventTimeExtractor) { public void collect(DoFn.ProcessContext ctx, OutputT out) { if (eventTimeExtractor != null) { InputT element = ctx.element(); - ctx.outputWithTimestamp(out, new Instant(eventTimeExtractor.extractTimestamp(element))); + ctx.outputWithTimestamp( + out, Instant.ofEpochMilli(eventTimeExtractor.extractTimestamp(element))); } else { ctx.output(out); } diff --git a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/ContiguousSequenceRangeTest.java b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/ContiguousSequenceRangeTest.java index cb4648cc8b65..563d37022473 100644 --- a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/ContiguousSequenceRangeTest.java +++ b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/ContiguousSequenceRangeTest.java @@ -31,25 +31,25 @@ public void testCompareTo() { assertEquals( "Empty range is smaller than another", -1, - ContiguousSequenceRange.EMPTY.compareTo(ContiguousSequenceRange.of(0, 5, new Instant()))); + ContiguousSequenceRange.EMPTY.compareTo(ContiguousSequenceRange.of(0, 5, Instant.now()))); assertEquals( "First range is smaller than the second", -1, - ContiguousSequenceRange.of(0, 2, new Instant()) - .compareTo(ContiguousSequenceRange.of(0, 5, new Instant()))); + ContiguousSequenceRange.of(0, 2, Instant.now()) + .compareTo(ContiguousSequenceRange.of(0, 5, Instant.now()))); assertEquals( "First range is larger than the second", 1, - ContiguousSequenceRange.of(0, 10, new Instant()) - .compareTo(ContiguousSequenceRange.of(0, 5, new Instant()))); + ContiguousSequenceRange.of(0, 10, Instant.now()) + .compareTo(ContiguousSequenceRange.of(0, 5, Instant.now()))); assertEquals( "Ranges are equal", 0, - ContiguousSequenceRange.of(0, 10, new Instant()) - .compareTo(ContiguousSequenceRange.of(0, 10, new Instant()))); + ContiguousSequenceRange.of(0, 10, Instant.now()) + .compareTo(ContiguousSequenceRange.of(0, 10, Instant.now()))); } public void testLargestRange() { @@ -59,9 +59,9 @@ public void testLargestRange() { ContiguousSequenceRange.largestRange(Collections.EMPTY_LIST)); ContiguousSequenceRange one = ContiguousSequenceRange.EMPTY; - ContiguousSequenceRange two = ContiguousSequenceRange.of(0, 5, new Instant()); - ContiguousSequenceRange three = ContiguousSequenceRange.of(0, 22, new Instant()); - ContiguousSequenceRange four = ContiguousSequenceRange.of(0, 10, new Instant()); + ContiguousSequenceRange two = ContiguousSequenceRange.of(0, 5, Instant.now()); + ContiguousSequenceRange three = ContiguousSequenceRange.of(0, 22, Instant.now()); + ContiguousSequenceRange four = ContiguousSequenceRange.of(0, 10, Instant.now()); assertEquals( "third range", three, diff --git a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java index 98bc7591f4d7..223c6a1e8780 100644 --- a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java +++ b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorGlobalSequenceTest.java @@ -74,7 +74,7 @@ public void testPerfectOrderingProcessing() throws CannotProvideCoderException { EMISSION_FREQUENCY_ON_EVERY_ELEMENT, INITIAL_SEQUENCE_OF_0, LARGE_MAX_RESULTS_PER_OUTPUT, - ContiguousSequenceRange.of(0, 6, new Instant())); + ContiguousSequenceRange.of(0, 6, Instant.now())); } @Test @@ -108,7 +108,7 @@ public void testOutOfSequenceProcessing() throws CannotProvideCoderException { EMISSION_FREQUENCY_ON_EVERY_ELEMENT, INITIAL_SEQUENCE_OF_0, LARGE_MAX_RESULTS_PER_OUTPUT, - ContiguousSequenceRange.of(0, 9, new Instant())); + ContiguousSequenceRange.of(0, 9, Instant.now())); } @Test @@ -147,7 +147,7 @@ public void testHandlingOfDuplicateSequences() throws CannotProvideCoderExceptio EMISSION_FREQUENCY_ON_EVERY_ELEMENT, INITIAL_SEQUENCE_OF_0, LARGE_MAX_RESULTS_PER_OUTPUT, - ContiguousSequenceRange.of(0, 4, new Instant())); + ContiguousSequenceRange.of(0, 4, Instant.now())); } @Test @@ -186,7 +186,7 @@ public void testTreatingSequencesBelowInitialAsDuplicates() throws CannotProvide EMISSION_FREQUENCY_ON_EVERY_ELEMENT, INITIAL_SEQUENCE_OF_0, LARGE_MAX_RESULTS_PER_OUTPUT, - ContiguousSequenceRange.of(0, 4, new Instant())); + ContiguousSequenceRange.of(0, 4, Instant.now())); } @Test @@ -224,7 +224,7 @@ public void testHandlingOfCheckedExceptions() throws CannotProvideCoderException // Sequence matcher doesn't know if the element is valid or not. // That's why the elements that are get rejected in the processor still count when // calculating the global sequence - ContiguousSequenceRange.of(0, 4, new Instant())); + ContiguousSequenceRange.of(0, 4, Instant.now())); } @Test @@ -251,7 +251,7 @@ public void testProcessingWithEveryOtherResultEmission() throws CannotProvideCod EMISSION_FREQUENCY_ON_EVERY_OTHER_EVENT, INITIAL_SEQUENCE_OF_0, LARGE_MAX_RESULTS_PER_OUTPUT, - ContiguousSequenceRange.of(0, 6, new Instant())); + ContiguousSequenceRange.of(0, 6, Instant.now())); } @Test @@ -285,7 +285,7 @@ public void testLargeBufferedOutputInTimer() throws CannotProvideCoderException EMISSION_FREQUENCY_ON_EVERY_ELEMENT, 1L /* This dataset assumes 1 as the starting sequence */, maxResultsPerOutput, - ContiguousSequenceRange.of(1, sequences.length + 1, new Instant())); + ContiguousSequenceRange.of(1, sequences.length + 1, Instant.now())); } @Test @@ -318,7 +318,7 @@ public void testSequenceGapProcessingInBufferedOutput() throws CannotProvideCode EMISSION_FREQUENCY_ON_EVERY_ELEMENT, 1L /* This dataset assumes 1 as the starting sequence */, maxResultsPerOutput, - ContiguousSequenceRange.of(1, 11, new Instant())); + ContiguousSequenceRange.of(1, 11, Instant.now())); } @Test @@ -374,7 +374,7 @@ public void testProcessingOfTheLastInput() throws CannotProvideCoderException { EMISSION_FREQUENCY_ON_EVERY_ELEMENT, INITIAL_SEQUENCE_OF_0, LARGE_MAX_RESULTS_PER_OUTPUT, - ContiguousSequenceRange.of(0, 3, new Instant())); + ContiguousSequenceRange.of(0, 3, Instant.now())); } private void testGlobalSequenceProcessing( @@ -443,7 +443,7 @@ private void testGlobalSequenceProcessing( @Test public void testWindowedProcessing() throws CannotProvideCoderException { - Instant base = new Instant(0); + Instant base = Instant.ofEpochMilli(0); TestStream values = TestStream.create(streamingPipeline.getCoderRegistry().getCoder(Event.class)) .advanceWatermarkTo(base) diff --git a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorPerKeySequenceTest.java b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorPerKeySequenceTest.java index 5dad7ac1852b..1f68ef7f9ce4 100644 --- a/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorPerKeySequenceTest.java +++ b/sdks/java/extensions/ordered/src/test/java/org/apache/beam/sdk/extensions/ordered/OrderedEventProcessorPerKeySequenceTest.java @@ -651,7 +651,7 @@ protected void testPerKeySequenceProcessing( @Test public void testWindowedProcessing() throws CannotProvideCoderException { - Instant base = new Instant(0); + Instant base = Instant.ofEpochMilli(0); TestStream values = TestStream.create(streamingPipeline.getCoderRegistry().getCoder(Event.class)) .advanceWatermarkTo(base) diff --git a/sdks/java/extensions/sql/iceberg/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/PubsubToIcebergIT.java b/sdks/java/extensions/sql/iceberg/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/PubsubToIcebergIT.java index 900fdae743a1..54d11d1dc350 100644 --- a/sdks/java/extensions/sql/iceberg/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/PubsubToIcebergIT.java +++ b/sdks/java/extensions/sql/iceberg/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/PubsubToIcebergIT.java @@ -293,6 +293,6 @@ private String jsonString(int id, String name) { } private Instant ts(long millis) { - return new Instant(millis); + return Instant.ofEpochMilli(millis); } } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestUnboundedTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestUnboundedTable.java index c18df6b0d3f8..a9a4f15e8e24 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestUnboundedTable.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestUnboundedTable.java @@ -111,7 +111,7 @@ public PCollection buildIOReader(PBegin begin) { TestStream.Builder values = TestStream.create(schema); for (Pair> pair : timestampedRows) { - values = values.advanceWatermarkTo(new Instant(0).plus(pair.getKey())); + values = values.advanceWatermarkTo(Instant.ofEpochMilli(0).plus(pair.getKey())); for (int i = 0; i < pair.getValue().size(); i++) { values = values.addElements( diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamComplexTypeTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamComplexTypeTest.java index a5f78f715293..689bf9ac460c 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamComplexTypeTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamComplexTypeTest.java @@ -479,7 +479,7 @@ public void testNullInnerRow() { @Test public void testDatetimeFields() { - Instant current = new Instant(1561671380000L); // Long value corresponds to 27/06/2019 + Instant current = Instant.ofEpochMilli(1561671380000L); // Long value corresponds to 27/06/2019 Schema dateTimeFieldSchema = Schema.builder() diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslUdfUdafTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslUdfUdafTest.java index 41288dd21e36..7669884777c3 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslUdfUdafTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslUdfUdafTest.java @@ -385,7 +385,7 @@ public Integer extractOutput(Integer accumulator) { public static class JodaMax extends CombineFn { @Override public Instant createAccumulator() { - return new Instant(0L); + return Instant.ofEpochMilli(0L); } @Override @@ -395,7 +395,7 @@ public Instant addInput(Instant accumulator, Instant input) { @Override public Instant mergeAccumulators(Iterable accumulators) { - Instant v = new Instant(0L); + Instant v = Instant.ofEpochMilli(0L); for (Instant accumulator : accumulators) { v = accumulator.isBefore(v) ? v : accumulator; } diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/PubsubToBigqueryIT.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/PubsubToBigqueryIT.java index 54be98adbc14..3160a6653418 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/PubsubToBigqueryIT.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/PubsubToBigqueryIT.java @@ -178,6 +178,6 @@ private String jsonString(int id, String name) { } private Instant ts(long millis) { - return new Instant(millis); + return Instant.ofEpochMilli(millis); } } diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java index 78aceeaab198..b3cfbc0a7830 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java @@ -1365,7 +1365,7 @@ private class FnApiTimer implements org.apache.beam.sdk.state.Timer { break; case PROCESSING_TIME: // TODO: This should use an injected clock when using TestStream. - fireTimestamp = new Instant(DateTimeUtils.currentTimeMillis()); + fireTimestamp = Instant.ofEpochMilli(DateTimeUtils.currentTimeMillis()); break; default: throw new IllegalArgumentException( diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/AssignWindowsRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/AssignWindowsRunnerTest.java index 78563dc24690..b0754ea507a1 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/AssignWindowsRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/AssignWindowsRunnerTest.java @@ -86,14 +86,14 @@ public void singleInputSingleOutputSucceeds() throws Exception { runner.assignWindows( WindowedValues.of( 2, - new Instant(-10L), - new IntervalWindow(new Instant(-120000L), Duration.standardMinutes(3L)), + Instant.ofEpochMilli(-10L), + new IntervalWindow(Instant.ofEpochMilli(-120000L), Duration.standardMinutes(3L)), PaneInfo.ON_TIME_AND_ONLY_FIRING)), equalTo( WindowedValues.of( 2, - new Instant(-10L), - windowFn.assignWindow(new Instant(-10L)), + Instant.ofEpochMilli(-10L), + windowFn.assignWindow(Instant.ofEpochMilli(-10L)), PaneInfo.ON_TIME_AND_ONLY_FIRING))); } @@ -106,27 +106,30 @@ public void singleInputMultipleOutputSucceeds() throws Exception { IntervalWindow firstWindow = new IntervalWindow( - new Instant(0).minus(Duration.standardMinutes(4L)), Duration.standardMinutes(4L)); + Instant.ofEpochMilli(0).minus(Duration.standardMinutes(4L)), + Duration.standardMinutes(4L)); IntervalWindow secondWindow = new IntervalWindow( - new Instant(0).minus(Duration.standardMinutes(2L)), Duration.standardMinutes(4L)); - IntervalWindow thirdWindow = new IntervalWindow(new Instant(0), Duration.standardMinutes(4L)); + Instant.ofEpochMilli(0).minus(Duration.standardMinutes(2L)), + Duration.standardMinutes(4L)); + IntervalWindow thirdWindow = + new IntervalWindow(Instant.ofEpochMilli(0), Duration.standardMinutes(4L)); WindowedValue firstValue = - WindowedValues.timestampedValueInGlobalWindow(-3, new Instant(-12)); + WindowedValues.timestampedValueInGlobalWindow(-3, Instant.ofEpochMilli(-12)); assertThat( runner.assignWindows(firstValue), equalTo( WindowedValues.of( -3, - new Instant(-12), + Instant.ofEpochMilli(-12), ImmutableSet.of(firstWindow, secondWindow), firstValue.getPaneInfo()))); WindowedValue secondValue = WindowedValues.of( 3, - new Instant(12), - new IntervalWindow(new Instant(-12), Duration.standardMinutes(24)), + Instant.ofEpochMilli(12), + new IntervalWindow(Instant.ofEpochMilli(-12), Duration.standardMinutes(24)), PaneInfo.ON_TIME_AND_ONLY_FIRING); assertThat( @@ -134,7 +137,7 @@ public void singleInputMultipleOutputSucceeds() throws Exception { equalTo( WindowedValues.of( 3, - new Instant(12), + Instant.ofEpochMilli(12), ImmutableSet.of(secondWindow, thirdWindow), secondValue.getPaneInfo()))); } @@ -148,7 +151,7 @@ public Collection assignWindows(AssignContext c) { c.window(); return ImmutableSet.of( GlobalWindow.INSTANCE, - new IntervalWindow(new Instant(-500), Duration.standardMinutes(3))); + new IntervalWindow(Instant.ofEpochMilli(-500), Duration.standardMinutes(3))); } @Override @@ -208,10 +211,10 @@ public Coder windowCoder() { WindowedValue value = WindowedValues.of( 2, - new Instant(-10L), + Instant.ofEpochMilli(-10L), ImmutableList.of( - new IntervalWindow(new Instant(-22L), Duration.standardMinutes(5L)), - new IntervalWindow(new Instant(-120000L), Duration.standardMinutes(3L))), + new IntervalWindow(Instant.ofEpochMilli(-22L), Duration.standardMinutes(5L)), + new IntervalWindow(Instant.ofEpochMilli(-120000L), Duration.standardMinutes(3L))), PaneInfo.ON_TIME_AND_ONLY_FIRING); context.getPCollectionConsumer("input").accept(value); assertThat( @@ -219,17 +222,17 @@ public Coder windowCoder() { containsInAnyOrder( WindowedValues.of( 2, - new Instant(-10L), + Instant.ofEpochMilli(-10L), ImmutableSet.of( GlobalWindow.INSTANCE, - new IntervalWindow(new Instant(-500), Duration.standardMinutes(3))), + new IntervalWindow(Instant.ofEpochMilli(-500), Duration.standardMinutes(3))), PaneInfo.ON_TIME_AND_ONLY_FIRING), WindowedValues.of( 2, - new Instant(-10L), + Instant.ofEpochMilli(-10L), ImmutableSet.of( GlobalWindow.INSTANCE, - new IntervalWindow(new Instant(-500), Duration.standardMinutes(3))), + new IntervalWindow(Instant.ofEpochMilli(-500), Duration.standardMinutes(3))), PaneInfo.ON_TIME_AND_ONLY_FIRING))); } @@ -268,10 +271,10 @@ public Coder windowCoder() { runner.assignWindows( WindowedValues.of( 2, - new Instant(-10L), + Instant.ofEpochMilli(-10L), ImmutableList.of( - new IntervalWindow(new Instant(-22L), Duration.standardMinutes(5L)), - new IntervalWindow(new Instant(-120000L), Duration.standardMinutes(3L))), + new IntervalWindow(Instant.ofEpochMilli(-22L), Duration.standardMinutes(5L)), + new IntervalWindow(Instant.ofEpochMilli(-120000L), Duration.standardMinutes(3L))), PaneInfo.ON_TIME_AND_ONLY_FIRING)); } @@ -303,14 +306,14 @@ public void factoryCreatesFromJavaWindowFn() throws Exception { fn.apply( WindowedValues.of( 22L, - new Instant(5), - new IntervalWindow(new Instant(0L), new Instant(20027L)), + Instant.ofEpochMilli(5), + new IntervalWindow(Instant.ofEpochMilli(0L), Instant.ofEpochMilli(20027L)), PaneInfo.ON_TIME_AND_ONLY_FIRING)), equalTo( WindowedValues.of( 22L, - new Instant(5), - new TestWindowFn().assignWindow(new Instant(5)), + Instant.ofEpochMilli(5), + new TestWindowFn().assignWindow(Instant.ofEpochMilli(5)), PaneInfo.ON_TIME_AND_ONLY_FIRING))); } @@ -341,8 +344,8 @@ public void factoryCreatesFromKnownWindowFn() throws Exception { fn.apply( WindowedValues.of( 22L, - new Instant(5), - new IntervalWindow(new Instant(0L), new Instant(20027L)), + Instant.ofEpochMilli(5), + new IntervalWindow(Instant.ofEpochMilli(0L), Instant.ofEpochMilli(20027L)), PaneInfo.ON_TIME_AND_ONLY_FIRING)); assertThat( @@ -350,8 +353,8 @@ public void factoryCreatesFromKnownWindowFn() throws Exception { equalTo( WindowedValues.of( 22L, - new Instant(5), - new IntervalWindow(new Instant(5L), Duration.standardMinutes(12L)), + Instant.ofEpochMilli(5), + new IntervalWindow(Instant.ofEpochMilli(5L), Duration.standardMinutes(12L)), PaneInfo.ON_TIME_AND_ONLY_FIRING))); } diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java index 50a2fec0b5a2..995a7500a2b5 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java @@ -556,36 +556,36 @@ public void testProcessElementWithNonWindowObservingOptimization() throws Except mainInput.accept( valueInWindows( "X", - new IntervalWindow(new Instant(0L), Duration.standardMinutes(1)), - new IntervalWindow(new Instant(10L), Duration.standardMinutes(1)))); + new IntervalWindow(Instant.ofEpochMilli(0L), Duration.standardMinutes(1)), + new IntervalWindow(Instant.ofEpochMilli(10L), Duration.standardMinutes(1)))); mainInput.accept( valueInWindows( "Y", - new IntervalWindow(new Instant(1000L), Duration.standardMinutes(1)), - new IntervalWindow(new Instant(1010L), Duration.standardMinutes(1)))); + new IntervalWindow(Instant.ofEpochMilli(1000L), Duration.standardMinutes(1)), + new IntervalWindow(Instant.ofEpochMilli(1010L), Duration.standardMinutes(1)))); // Ensure that each output element is in all the windows and not one per window. assertThat( mainOutputValues, contains( valueInWindows( "X:main", - new IntervalWindow(new Instant(0L), Duration.standardMinutes(1)), - new IntervalWindow(new Instant(10L), Duration.standardMinutes(1))), + new IntervalWindow(Instant.ofEpochMilli(0L), Duration.standardMinutes(1)), + new IntervalWindow(Instant.ofEpochMilli(10L), Duration.standardMinutes(1))), valueInWindows( "Y:main", - new IntervalWindow(new Instant(1000L), Duration.standardMinutes(1)), - new IntervalWindow(new Instant(1010L), Duration.standardMinutes(1))))); + new IntervalWindow(Instant.ofEpochMilli(1000L), Duration.standardMinutes(1)), + new IntervalWindow(Instant.ofEpochMilli(1010L), Duration.standardMinutes(1))))); assertThat( additionalOutputValues, contains( valueInWindows( "X:additional", - new IntervalWindow(new Instant(0L), Duration.standardMinutes(1)), - new IntervalWindow(new Instant(10L), Duration.standardMinutes(1))), + new IntervalWindow(Instant.ofEpochMilli(0L), Duration.standardMinutes(1)), + new IntervalWindow(Instant.ofEpochMilli(10L), Duration.standardMinutes(1))), valueInWindows( "Y:additional", - new IntervalWindow(new Instant(1000L), Duration.standardMinutes(1)), - new IntervalWindow(new Instant(1010L), Duration.standardMinutes(1))))); + new IntervalWindow(Instant.ofEpochMilli(1000L), Duration.standardMinutes(1)), + new IntervalWindow(Instant.ofEpochMilli(1010L), Duration.standardMinutes(1))))); mainOutputValues.clear(); Iterables.getOnlyElement(context.getFinishBundleFunctions()).run(); @@ -619,8 +619,8 @@ public void processElement(ProcessContext context) { @Test public void testSideInputIsAccessibleForDownstreamCallers() throws Exception { FixedWindows windowFn = FixedWindows.of(Duration.millis(1L)); - IntervalWindow windowA = windowFn.assignWindow(new Instant(1L)); - IntervalWindow windowB = windowFn.assignWindow(new Instant(2L)); + IntervalWindow windowA = windowFn.assignWindow(Instant.ofEpochMilli(1L)); + IntervalWindow windowB = windowFn.assignWindow(Instant.ofEpochMilli(2L)); ByteString encodedWindowA = ByteString.copyFrom(CoderUtils.encodeToByteArray(windowFn.windowCoder(), windowA)); ByteString encodedWindowB = @@ -730,8 +730,8 @@ public void testUsingMetrics() throws Exception { metricsContainerRegistry.getContainer(TEST_TRANSFORM_ID); Closeable closeable = MetricsEnvironment.scopedMetricsContainer(metricsContainer); FixedWindows windowFn = FixedWindows.of(Duration.millis(1L)); - IntervalWindow windowA = windowFn.assignWindow(new Instant(1L)); - IntervalWindow windowB = windowFn.assignWindow(new Instant(2L)); + IntervalWindow windowA = windowFn.assignWindow(Instant.ofEpochMilli(1L)); + IntervalWindow windowB = windowFn.assignWindow(Instant.ofEpochMilli(2L)); ByteString encodedWindowA = ByteString.copyFrom(CoderUtils.encodeToByteArray(windowFn.windowCoder(), windowA)); ByteString encodedWindowB = @@ -959,81 +959,101 @@ public void testTimers() throws Exception { // Ensure that the key order does not matter when we traverse over KV pairs. FnDataReceiver> mainInput = context.getPCollectionConsumer(inputPCollectionId); - mainInput.accept(timestampedValueInGlobalWindow(KV.of("X", "X1"), new Instant(1000L))); - mainInput.accept(timestampedValueInGlobalWindow(KV.of("Y", "Y1"), new Instant(1100L))); - mainInput.accept(timestampedValueInGlobalWindow(KV.of("X", "X2"), new Instant(1200L))); - mainInput.accept(timestampedValueInGlobalWindow(KV.of("Y", "Y2"), new Instant(1300L))); + mainInput.accept( + timestampedValueInGlobalWindow(KV.of("X", "X1"), Instant.ofEpochMilli(1000L))); + mainInput.accept( + timestampedValueInGlobalWindow(KV.of("Y", "Y1"), Instant.ofEpochMilli(1100L))); + mainInput.accept( + timestampedValueInGlobalWindow(KV.of("X", "X2"), Instant.ofEpochMilli(1200L))); + mainInput.accept( + timestampedValueInGlobalWindow(KV.of("Y", "Y2"), Instant.ofEpochMilli(1300L))); context .getIncomingTimerEndpoint(eventTimer.getTimerFamilyId()) .getReceiver() - .accept(timerInGlobalWindow("A", new Instant(1400L), new Instant(2400L))); + .accept( + timerInGlobalWindow("A", Instant.ofEpochMilli(1400L), Instant.ofEpochMilli(2400L))); context .getIncomingTimerEndpoint(eventTimer.getTimerFamilyId()) .getReceiver() - .accept(timerInGlobalWindow("B", new Instant(1500L), new Instant(2500L))); + .accept( + timerInGlobalWindow("B", Instant.ofEpochMilli(1500L), Instant.ofEpochMilli(2500L))); // This will be ignored since there are earlier timers, and the earlier timer will eventually // push the timer past 1600L. context .getIncomingTimerEndpoint(eventTimer.getTimerFamilyId()) .getReceiver() - .accept(timerInGlobalWindow("A", new Instant(1600L), new Instant(2600L))); + .accept( + timerInGlobalWindow("A", Instant.ofEpochMilli(1600L), Instant.ofEpochMilli(2600L))); // This will be ignored since the timer was already cleared in this bundle. context .getIncomingTimerEndpoint(processingTimer.getTimerFamilyId()) .getReceiver() - .accept(timerInGlobalWindow("X", new Instant(1700L), new Instant(2700L))); + .accept( + timerInGlobalWindow("X", Instant.ofEpochMilli(1700L), Instant.ofEpochMilli(2700L))); context .getIncomingTimerEndpoint(processingTimer.getTimerFamilyId()) .getReceiver() - .accept(timerInGlobalWindow("C", new Instant(1800L), new Instant(2800L))); + .accept( + timerInGlobalWindow("C", Instant.ofEpochMilli(1800L), Instant.ofEpochMilli(2800L))); context .getIncomingTimerEndpoint(processingTimer.getTimerFamilyId()) .getReceiver() - .accept(timerInGlobalWindow("B", new Instant(1500), new Instant(10032))); + .accept( + timerInGlobalWindow("B", Instant.ofEpochMilli(1500), Instant.ofEpochMilli(10032))); context .getIncomingTimerEndpoint(eventFamilyTimer.getTimerFamilyId()) .getReceiver() .accept( dynamicTimerInGlobalWindow( - "B", "event-timer2", new Instant(2000L), new Instant(1650L))); + "B", "event-timer2", Instant.ofEpochMilli(2000L), Instant.ofEpochMilli(1650L))); context .getIncomingTimerEndpoint(processingFamilyTimer.getTimerFamilyId()) .getReceiver() .accept( dynamicTimerInGlobalWindow( - "Y", "processing-timer2", new Instant(2100L), new Instant(3100L))); + "Y", + "processing-timer2", + Instant.ofEpochMilli(2100L), + Instant.ofEpochMilli(3100L))); assertThat( - mainOutputValues.get(0), isValueInGlobalWindow("key:X mainX[X0]", new Instant(1000L))); + mainOutputValues.get(0), + isValueInGlobalWindow("key:X mainX[X0]", Instant.ofEpochMilli(1000L))); assertThat( mainOutputValues, containsInAnyOrder( - isValueInGlobalWindow("key:X mainX[X0]", new Instant(1000L)), - isValueInGlobalWindow("key:Y mainY[]", new Instant(1100L)), - isValueInGlobalWindow("key:X mainX[X0, X1]", new Instant(1200L)), - isValueInGlobalWindow("key:Y mainY[Y1]", new Instant(1300L)), - isValueInGlobalWindow("key:A event[A0]", new Instant(1400L)), - isValueInGlobalWindow("key:B event[]", new Instant(1500L)), - isValueInGlobalWindow("key:A event[A0, event]", new Instant(1400L)), - isValueInGlobalWindow("key:A event[A0, event, event]", new Instant(1400L)), - isValueInGlobalWindow("key:A event[A0, event, event, event]", new Instant(1400L)), + isValueInGlobalWindow("key:X mainX[X0]", Instant.ofEpochMilli(1000L)), + isValueInGlobalWindow("key:Y mainY[]", Instant.ofEpochMilli(1100L)), + isValueInGlobalWindow("key:X mainX[X0, X1]", Instant.ofEpochMilli(1200L)), + isValueInGlobalWindow("key:Y mainY[Y1]", Instant.ofEpochMilli(1300L)), + isValueInGlobalWindow("key:A event[A0]", Instant.ofEpochMilli(1400L)), + isValueInGlobalWindow("key:B event[]", Instant.ofEpochMilli(1500L)), + isValueInGlobalWindow("key:A event[A0, event]", Instant.ofEpochMilli(1400L)), + isValueInGlobalWindow("key:A event[A0, event, event]", Instant.ofEpochMilli(1400L)), + isValueInGlobalWindow( + "key:A event[A0, event, event, event]", Instant.ofEpochMilli(1400L)), isValueInGlobalWindow( - "key:A event[A0, event, event, event, event]", new Instant(1400L)), + "key:A event[A0, event, event, event, event]", Instant.ofEpochMilli(1400L)), isValueInGlobalWindow( - "key:A event[A0, event, event, event, event, event]", new Instant(1400L)), + "key:A event[A0, event, event, event, event, event]", + Instant.ofEpochMilli(1400L)), isValueInGlobalWindow( - "key:A event[A0, event, event, event, event, event, event]", new Instant(1400L)), - isValueInGlobalWindow("key:C processing[C0]", new Instant(1800L)), - isValueInGlobalWindow("key:B processing[event]", new Instant(1500L)), - isValueInGlobalWindow("key:B event[event, processing]", new Instant(1500)), - isValueInGlobalWindow("key:B event[event, processing, event]", new Instant(1500)), + "key:A event[A0, event, event, event, event, event, event]", + Instant.ofEpochMilli(1400L)), + isValueInGlobalWindow("key:C processing[C0]", Instant.ofEpochMilli(1800L)), + isValueInGlobalWindow("key:B processing[event]", Instant.ofEpochMilli(1500L)), + isValueInGlobalWindow("key:B event[event, processing]", Instant.ofEpochMilli(1500)), isValueInGlobalWindow( - "key:B event[event, processing, event, event]", new Instant(1500)), + "key:B event[event, processing, event]", Instant.ofEpochMilli(1500)), isValueInGlobalWindow( - "key:B event-family[event, processing, event, event, event]", new Instant(2000L)), - isValueInGlobalWindow("key:Y processing-family[Y1, Y2]", new Instant(2100L)))); + "key:B event[event, processing, event, event]", Instant.ofEpochMilli(1500)), + isValueInGlobalWindow( + "key:B event-family[event, processing, event, event, event]", + Instant.ofEpochMilli(2000L)), + isValueInGlobalWindow( + "key:Y processing-family[Y1, Y2]", Instant.ofEpochMilli(2100L)))); mainOutputValues.clear(); @@ -1044,49 +1064,64 @@ public void testTimers() throws Exception { aggregator.getOutputTimers().get(eventTimer), contains( clearedTimerInGlobalWindow("X"), - timerInGlobalWindow("Y", new Instant(2100L), new Instant(2181L)), - timerInGlobalWindow("A", new Instant(1400L), new Instant(2617L)), - timerInGlobalWindow("B", new Instant(2000L), new Instant(2071L)), - timerInGlobalWindow("C", new Instant(1800L), new Instant(1861L)))); + timerInGlobalWindow("Y", Instant.ofEpochMilli(2100L), Instant.ofEpochMilli(2181L)), + timerInGlobalWindow("A", Instant.ofEpochMilli(1400L), Instant.ofEpochMilli(2617L)), + timerInGlobalWindow("B", Instant.ofEpochMilli(2000L), Instant.ofEpochMilli(2071L)), + timerInGlobalWindow("C", Instant.ofEpochMilli(1800L), Instant.ofEpochMilli(1861L)))); assertThat( aggregator.getOutputTimers().get(processingTimer), contains( clearedTimerInGlobalWindow("X"), - timerInGlobalWindow("Y", new Instant(2100L), new Instant(10082L)), - timerInGlobalWindow("A", new Instant(1400L), new Instant(10032L)), - timerInGlobalWindow("B", new Instant(2000L), new Instant(10072L)), - timerInGlobalWindow("C", new Instant(1800L), new Instant(10062L)))); + timerInGlobalWindow("Y", Instant.ofEpochMilli(2100L), Instant.ofEpochMilli(10082L)), + timerInGlobalWindow("A", Instant.ofEpochMilli(1400L), Instant.ofEpochMilli(10032L)), + timerInGlobalWindow("B", Instant.ofEpochMilli(2000L), Instant.ofEpochMilli(10072L)), + timerInGlobalWindow("C", Instant.ofEpochMilli(1800L), Instant.ofEpochMilli(10062L)))); assertThat( aggregator.getOutputTimers().get(eventFamilyTimer), containsInAnyOrder( dynamicTimerInGlobalWindow( - "X", "event-timer1", new Instant(1200L), new Instant(1203L)), + "X", "event-timer1", Instant.ofEpochMilli(1200L), Instant.ofEpochMilli(1203L)), clearedTimerInGlobalWindow("X", "to-delete-event"), clearedTimerInGlobalWindow("Y", "to-delete-event"), dynamicTimerInGlobalWindow( - "Y", "event-timer1", new Instant(2100L), new Instant(2183L)), + "Y", "event-timer1", Instant.ofEpochMilli(2100L), Instant.ofEpochMilli(2183L)), dynamicTimerInGlobalWindow( - "A", "event-timer1", new Instant(1400L), new Instant(2619L)), + "A", "event-timer1", Instant.ofEpochMilli(1400L), Instant.ofEpochMilli(2619L)), dynamicTimerInGlobalWindow( - "B", "event-timer1", new Instant(2000L), new Instant(2073L)), + "B", "event-timer1", Instant.ofEpochMilli(2000L), Instant.ofEpochMilli(2073L)), dynamicTimerInGlobalWindow( - "C", "event-timer1", new Instant(1800L), new Instant(1863L)))); + "C", "event-timer1", Instant.ofEpochMilli(1800L), Instant.ofEpochMilli(1863L)))); assertThat( aggregator.getOutputTimers().get(processingFamilyTimer), containsInAnyOrder( dynamicTimerInGlobalWindow( - "X", "processing-timer1", new Instant(1200L), new Instant(10004L)), + "X", + "processing-timer1", + Instant.ofEpochMilli(1200L), + Instant.ofEpochMilli(10004L)), clearedTimerInGlobalWindow("X", "to-delete-processing"), dynamicTimerInGlobalWindow( - "Y", "processing-timer1", new Instant(2100L), new Instant(10084L)), + "Y", + "processing-timer1", + Instant.ofEpochMilli(2100L), + Instant.ofEpochMilli(10084L)), clearedTimerInGlobalWindow("Y", "to-delete-processing"), dynamicTimerInGlobalWindow( - "A", "processing-timer1", new Instant(1400L), new Instant(10034L)), + "A", + "processing-timer1", + Instant.ofEpochMilli(1400L), + Instant.ofEpochMilli(10034L)), dynamicTimerInGlobalWindow( - "B", "processing-timer1", new Instant(2000L), new Instant(10074L)), + "B", + "processing-timer1", + Instant.ofEpochMilli(2000L), + Instant.ofEpochMilli(10074L)), dynamicTimerInGlobalWindow( - "C", "processing-timer1", new Instant(1800L), new Instant(10064L)))); + "C", + "processing-timer1", + Instant.ofEpochMilli(1800L), + Instant.ofEpochMilli(10064L)))); assertThat(mainOutputValues, empty()); @@ -2196,8 +2231,8 @@ public void testProcessElementForWindowedSizedElementAndRestriction() throws Exc context.getPCollectionConsumer(inputPCollectionId); assertThat(mainInput, instanceOf(HandlesSplits.class)); - BoundedWindow window1 = new IntervalWindow(new Instant(5), new Instant(10)); - BoundedWindow window2 = new IntervalWindow(new Instant(6), new Instant(11)); + BoundedWindow window1 = new IntervalWindow(Instant.ofEpochMilli(5), Instant.ofEpochMilli(10)); + BoundedWindow window2 = new IntervalWindow(Instant.ofEpochMilli(6), Instant.ofEpochMilli(11)); { // Check that before processing an element we don't report progress assertNoReportedProgress(context.getBundleProgressReporters()); @@ -2687,7 +2722,7 @@ public void testDoFnSkewNotAllowed() throws Exception { assertThrows( UserCodeException.class, () -> { - mainInput.accept(timestampedValueInGlobalWindow("1", new Instant(0L))); + mainInput.accept(timestampedValueInGlobalWindow("1", Instant.ofEpochMilli(0L))); }) .getMessage(); @@ -2695,7 +2730,8 @@ public void testDoFnSkewNotAllowed() throws Exception { message, allOf( containsString( - String.format("timestamp %s", new Instant(0).minus(Duration.millis(1L)))), + String.format( + "timestamp %s", Instant.ofEpochMilli(0).minus(Duration.millis(1L)))), containsString( String.format( "allowed skew (%s)", @@ -2744,7 +2780,7 @@ public void testDoFnSkewAllowed() throws Exception { FnDataReceiver> mainInput = context.getPCollectionConsumer(inputPCollectionId); mainInput.accept(valueInGlobalWindow("0")); - mainInput.accept(timestampedValueInGlobalWindow("3", new Instant(0L))); + mainInput.accept(timestampedValueInGlobalWindow("3", Instant.ofEpochMilli(0L))); } } diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/MapFnRunnersTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/MapFnRunnersTest.java index e881c35ce710..0c0ae6ec1f05 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/MapFnRunnersTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/MapFnRunnersTest.java @@ -148,24 +148,25 @@ public void testFullWindowedValueMappingWithCompressedWindow() throws Exception assertThat( context.getPCollectionConsumers().keySet(), containsInAnyOrder("inputPC", "outputPC")); - IntervalWindow firstWindow = new IntervalWindow(new Instant(0L), Duration.standardMinutes(10L)); + IntervalWindow firstWindow = + new IntervalWindow(Instant.ofEpochMilli(0L), Duration.standardMinutes(10L)); IntervalWindow secondWindow = - new IntervalWindow(new Instant(-10L), Duration.standardSeconds(22L)); + new IntervalWindow(Instant.ofEpochMilli(-10L), Duration.standardSeconds(22L)); context .getPCollectionConsumer("inputPC") .accept( WindowedValues.of( "abc", - new Instant(12), + Instant.ofEpochMilli(12), ImmutableSet.of(firstWindow, GlobalWindow.INSTANCE, secondWindow), PaneInfo.NO_FIRING)); assertThat( outputConsumer, containsInAnyOrder( - WindowedValues.timestampedValueInGlobalWindow("ABC", new Instant(12)), - WindowedValues.of("ABC", new Instant(12), secondWindow, PaneInfo.NO_FIRING), - WindowedValues.of("ABC", new Instant(12), firstWindow, PaneInfo.NO_FIRING))); + WindowedValues.timestampedValueInGlobalWindow("ABC", Instant.ofEpochMilli(12)), + WindowedValues.of("ABC", Instant.ofEpochMilli(12), secondWindow, PaneInfo.NO_FIRING), + WindowedValues.of("ABC", Instant.ofEpochMilli(12), firstWindow, PaneInfo.NO_FIRING))); } public ThrowingFunction, WindowedValue> diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/PrecombineGroupingTableTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/PrecombineGroupingTableTest.java index 384f46bb4527..6b062d131e74 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/PrecombineGroupingTableTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/PrecombineGroupingTableTest.java @@ -124,17 +124,17 @@ public void testCombiningInheritsOneOfTheValuesTimestamps() throws Exception { TestOutputReceiver>> receiver = new TestOutputReceiver<>(); - table.put(timestampedValueInGlobalWindow(KV.of("A", 1), new Instant(1)), receiver); - table.put(timestampedValueInGlobalWindow(KV.of("B", 9), new Instant(21)), receiver); - table.put(timestampedValueInGlobalWindow(KV.of("A", 2), new Instant(1)), receiver); - table.put(timestampedValueInGlobalWindow(KV.of("B", 2), new Instant(20)), receiver); - table.put(timestampedValueInGlobalWindow(KV.of("A", 4), new Instant(1)), receiver); + table.put(timestampedValueInGlobalWindow(KV.of("A", 1), Instant.ofEpochMilli(1)), receiver); + table.put(timestampedValueInGlobalWindow(KV.of("B", 9), Instant.ofEpochMilli(21)), receiver); + table.put(timestampedValueInGlobalWindow(KV.of("A", 2), Instant.ofEpochMilli(1)), receiver); + table.put(timestampedValueInGlobalWindow(KV.of("B", 2), Instant.ofEpochMilli(20)), receiver); + table.put(timestampedValueInGlobalWindow(KV.of("A", 4), Instant.ofEpochMilli(1)), receiver); table.flush(receiver); assertThat( receiver.outputElems, containsInAnyOrder( - timestampedValueInGlobalWindow(KV.of("A", 1L + 2 + 4), new Instant(1)), - timestampedValueInGlobalWindow(KV.of("B", 9L + 2), new Instant(21)))); + timestampedValueInGlobalWindow(KV.of("A", 1L + 2 + 4), Instant.ofEpochMilli(1)), + timestampedValueInGlobalWindow(KV.of("B", 9L + 2), Instant.ofEpochMilli(21)))); } @Test diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/SplittablePairWithRestrictionDoFnRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/SplittablePairWithRestrictionDoFnRunnerTest.java index 756f17fdfa32..69f4c2a5dc2b 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/SplittablePairWithRestrictionDoFnRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/SplittablePairWithRestrictionDoFnRunnerTest.java @@ -347,8 +347,10 @@ public void testProcessElementForWindowedPairWithRestriction() throws Exception FnDataReceiver> mainInput = context.getPCollectionConsumer(inputPCollectionId); - IntervalWindow window1 = new IntervalWindow(new Instant(5), new Instant(10)); - IntervalWindow window2 = new IntervalWindow(new Instant(6), new Instant(11)); + IntervalWindow window1 = + new IntervalWindow(Instant.ofEpochMilli(5), Instant.ofEpochMilli(10)); + IntervalWindow window2 = + new IntervalWindow(Instant.ofEpochMilli(6), Instant.ofEpochMilli(11)); WindowedValue firstValue = valueInWindows("5", window1, window2); WindowedValue secondValue = valueInWindows("2", window1, window2); mainInput.accept(firstValue); @@ -461,8 +463,10 @@ public void testProcessElementForWindowedPairWithRestrictionWithNonWindowObservi FnDataReceiver> mainInput = context.getPCollectionConsumer(inputPCollectionId); - IntervalWindow window1 = new IntervalWindow(new Instant(5), new Instant(10)); - IntervalWindow window2 = new IntervalWindow(new Instant(6), new Instant(11)); + IntervalWindow window1 = + new IntervalWindow(Instant.ofEpochMilli(5), Instant.ofEpochMilli(10)); + IntervalWindow window2 = + new IntervalWindow(Instant.ofEpochMilli(6), Instant.ofEpochMilli(11)); WindowedValue firstValue = valueInWindows("5", window1, window2); WindowedValue secondValue = valueInWindows("2", window1, window2); mainInput.accept(firstValue); diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/SplittableSplitAndSizeRestrictionsDoFnRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/SplittableSplitAndSizeRestrictionsDoFnRunnerTest.java index 34ef3e95b191..fe8ce71c88d5 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/SplittableSplitAndSizeRestrictionsDoFnRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/SplittableSplitAndSizeRestrictionsDoFnRunnerTest.java @@ -308,8 +308,8 @@ public void testProcessElementForWindowedSplitAndSizeRestriction() throws Except containsInAnyOrder(inputPCollectionId, outputPCollectionId)); FnDataReceiver> mainInput = context.getPCollectionConsumer(inputPCollectionId); - IntervalWindow window1 = new IntervalWindow(new Instant(5), new Instant(10)); - IntervalWindow window2 = new IntervalWindow(new Instant(6), new Instant(11)); + IntervalWindow window1 = new IntervalWindow(Instant.ofEpochMilli(5), Instant.ofEpochMilli(10)); + IntervalWindow window2 = new IntervalWindow(Instant.ofEpochMilli(6), Instant.ofEpochMilli(11)); WindowedValue firstValue = valueInWindows( KV.of("5", KV.of(new OffsetRange(0, 5), GlobalWindow.TIMESTAMP_MIN_VALUE)), @@ -456,8 +456,8 @@ public void testProcessElementForWindowedSplitAndSizeRestriction() throws Except containsInAnyOrder(inputPCollectionId, outputPCollectionId)); FnDataReceiver> mainInput = context.getPCollectionConsumer(inputPCollectionId); - IntervalWindow window1 = new IntervalWindow(new Instant(5), new Instant(10)); - IntervalWindow window2 = new IntervalWindow(new Instant(6), new Instant(11)); + IntervalWindow window1 = new IntervalWindow(Instant.ofEpochMilli(5), Instant.ofEpochMilli(10)); + IntervalWindow window2 = new IntervalWindow(Instant.ofEpochMilli(6), Instant.ofEpochMilli(11)); WindowedValue firstValue = valueInWindows( KV.of("5", KV.of(new OffsetRange(0, 5), GlobalWindow.TIMESTAMP_MIN_VALUE)), diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/SplittableTruncateSizedRestrictionsDoFnRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/SplittableTruncateSizedRestrictionsDoFnRunnerTest.java index 615a681095e1..6d82894eabed 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/SplittableTruncateSizedRestrictionsDoFnRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/SplittableTruncateSizedRestrictionsDoFnRunnerTest.java @@ -277,9 +277,9 @@ public void testProcessElementForTruncateAndSizeRestrictionForwardSplitWhenObser assertThat(mainInput, instanceOf(HandlesSplits.class)); mainOutputValues.clear(); - BoundedWindow window1 = new IntervalWindow(new Instant(5), new Instant(10)); - BoundedWindow window2 = new IntervalWindow(new Instant(6), new Instant(11)); - BoundedWindow window3 = new IntervalWindow(new Instant(7), new Instant(12)); + BoundedWindow window1 = new IntervalWindow(Instant.ofEpochMilli(5), Instant.ofEpochMilli(10)); + BoundedWindow window2 = new IntervalWindow(Instant.ofEpochMilli(6), Instant.ofEpochMilli(11)); + BoundedWindow window3 = new IntervalWindow(Instant.ofEpochMilli(7), Instant.ofEpochMilli(12)); // Setup and launch the trySplit thread. ExecutorService executorService = Executors.newSingleThreadExecutor(); Future trySplitFuture = @@ -588,8 +588,10 @@ public void testProcessElementForWindowedTruncateAndSizeRestriction() throws Exc context.getPCollectionConsumer(inputPCollectionId); assertThat(mainInput, instanceOf(HandlesSplits.class)); - IntervalWindow window1 = new IntervalWindow(new Instant(5), new Instant(10)); - IntervalWindow window2 = new IntervalWindow(new Instant(6), new Instant(11)); + IntervalWindow window1 = + new IntervalWindow(Instant.ofEpochMilli(5), Instant.ofEpochMilli(10)); + IntervalWindow window2 = + new IntervalWindow(Instant.ofEpochMilli(6), Instant.ofEpochMilli(11)); WindowedValue firstValue = valueInWindows( KV.of( @@ -705,8 +707,10 @@ public void testProcessElementForWindowedTruncateAndSizeRestriction() throws Exc context.getPCollectionConsumer(inputPCollectionId); assertThat(mainInput, instanceOf(HandlesSplits.class)); - IntervalWindow window1 = new IntervalWindow(new Instant(5), new Instant(10)); - IntervalWindow window2 = new IntervalWindow(new Instant(6), new Instant(11)); + IntervalWindow window1 = + new IntervalWindow(Instant.ofEpochMilli(5), Instant.ofEpochMilli(10)); + IntervalWindow window2 = + new IntervalWindow(Instant.ofEpochMilli(6), Instant.ofEpochMilli(11)); WindowedValue firstValue = valueInWindows( KV.of( diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/WindowMergingFnRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/WindowMergingFnRunnerTest.java index 2deaa602c252..df7d4c2de0db 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/WindowMergingFnRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/WindowMergingFnRunnerTest.java @@ -81,14 +81,14 @@ public void testWindowMergingWithMergingWindowFn() throws Exception { // 7, 8 and 10 should all be merged. 1 and 20 should remain in the original set. BoundedWindow[] expectedToBeMerged = new BoundedWindow[] { - new IntervalWindow(new Instant(9L), new Instant(11L)), - new IntervalWindow(new Instant(10L), new Instant(10L)), - new IntervalWindow(new Instant(7L), new Instant(10L)) + new IntervalWindow(Instant.ofEpochMilli(9L), Instant.ofEpochMilli(11L)), + new IntervalWindow(Instant.ofEpochMilli(10L), Instant.ofEpochMilli(10L)), + new IntervalWindow(Instant.ofEpochMilli(7L), Instant.ofEpochMilli(10L)) }; Iterable expectedToBeUnmerged = Sets.newHashSet( - new IntervalWindow(new Instant(1L), new Instant(1L)), - new IntervalWindow(new Instant(20L), new Instant(20L))); + new IntervalWindow(Instant.ofEpochMilli(1L), Instant.ofEpochMilli(1L)), + new IntervalWindow(Instant.ofEpochMilli(20L), Instant.ofEpochMilli(20L))); KV> input = KV.of( "abc", @@ -103,14 +103,16 @@ public void testWindowMergingWithMergingWindowFn() throws Exception { assertEquals(expectedToBeUnmerged, output.getValue().getKey()); KV> mergedOutput = Iterables.getOnlyElement(output.getValue().getValue()); - assertEquals(new IntervalWindow(new Instant(7L), new Instant(11L)), mergedOutput.getKey()); + assertEquals( + new IntervalWindow(Instant.ofEpochMilli(7L), Instant.ofEpochMilli(11L)), + mergedOutput.getKey()); assertThat(mergedOutput.getValue(), containsInAnyOrder(expectedToBeMerged)); // Process a new group of windows, make sure that previous result has been cleaned up. BoundedWindow[] expectedToBeMergedGroup2 = new BoundedWindow[] { - new IntervalWindow(new Instant(15L), new Instant(17L)), - new IntervalWindow(new Instant(16L), new Instant(18L)) + new IntervalWindow(Instant.ofEpochMilli(15L), Instant.ofEpochMilli(17L)), + new IntervalWindow(Instant.ofEpochMilli(16L), Instant.ofEpochMilli(18L)) }; input = @@ -125,7 +127,9 @@ public void testWindowMergingWithMergingWindowFn() throws Exception { assertEquals(input.getKey(), output.getKey()); assertEquals(expectedToBeUnmerged, output.getValue().getKey()); mergedOutput = Iterables.getOnlyElement(output.getValue().getValue()); - assertEquals(new IntervalWindow(new Instant(15L), new Instant(18L)), mergedOutput.getKey()); + assertEquals( + new IntervalWindow(Instant.ofEpochMilli(15L), Instant.ofEpochMilli(18L)), + mergedOutput.getKey()); assertThat(mergedOutput.getValue(), containsInAnyOrder(expectedToBeMergedGroup2)); } diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisReader.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisReader.java index 2cf9c9669caa..9d246ae69c5a 100644 --- a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisReader.java +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisReader.java @@ -49,7 +49,7 @@ class KinesisReader extends UnboundedSource.UnboundedReader { private final Duration backlogBytesCheckThreshold; private CustomOptional currentRecord = CustomOptional.absent(); private long lastBacklogBytes; - private Instant backlogBytesLastCheckTime = new Instant(0L); + private Instant backlogBytesLastCheckTime = Instant.ofEpochMilli(0L); private ShardReadersPool shardReadersPool; KinesisReader( diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/SimplifiedKinesisClient.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/SimplifiedKinesisClient.java index d93252885171..4e212e1f8017 100644 --- a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/SimplifiedKinesisClient.java +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/SimplifiedKinesisClient.java @@ -166,7 +166,7 @@ public static List deaggregate(List records) { */ public long getBacklogBytes(String streamName, Instant countSince) throws TransientKinesisException { - return getBacklogBytes(streamName, countSince, new Instant()); + return getBacklogBytes(streamName, countSince, Instant.now()); } /** diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/TimeUtil.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/TimeUtil.java index e9c5573169ef..3c09cd6b0846 100644 --- a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/TimeUtil.java +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/TimeUtil.java @@ -36,7 +36,9 @@ public static java.time.Instant toJava(Instant timestamp) { } public static Instant toJoda(java.time.Instant instant) { - return Optional.ofNullable(instant).map(t -> new Instant(t.toEpochMilli())).orElse(null); + return Optional.ofNullable(instant) + .map(t -> Instant.ofEpochMilli(t.toEpochMilli())) + .orElse(null); } static Instant minTimestamp(Stream timestamps) { diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/SqsUnboundedReader.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/SqsUnboundedReader.java index dd9050886887..0cdff5af355d 100644 --- a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/SqsUnboundedReader.java +++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/SqsUnboundedReader.java @@ -396,7 +396,7 @@ public Instant getWatermark() { // else: We're not confident enough to estimate a new watermark. Stick with the old one. minWatermarkMsSinceEpoch.add(nowMsSinceEpoch, lastWatermarkMsSinceEpoch); maxWatermarkMsSinceEpoch.add(nowMsSinceEpoch, lastWatermarkMsSinceEpoch); - return new Instant(lastWatermarkMsSinceEpoch); + return Instant.ofEpochMilli(lastWatermarkMsSinceEpoch); } @Override @@ -413,7 +413,7 @@ public Instant getCurrentTimestamp() throws NoSuchElementException { throw new NoSuchElementException(); } - return new Instant(current.getTimeStamp()); + return Instant.ofEpochMilli(current.getTimeStamp()); } @Override @@ -946,12 +946,12 @@ private void stats() { messageSkew, watermarkSkew, numLateMessages.get(nowMsSinceEpoch), - new Instant(lastWatermarkMsSinceEpoch), - new Instant(minReadTimestampMsSinceEpoch.get(nowMsSinceEpoch)), + Instant.ofEpochMilli(lastWatermarkMsSinceEpoch), + Instant.ofEpochMilli(minReadTimestampMsSinceEpoch.get(nowMsSinceEpoch)), minReadTimestampMsSinceEpoch.isSignificant(), - new Instant(minUnreadTimestampMsSinceEpoch.get()), + Instant.ofEpochMilli(minUnreadTimestampMsSinceEpoch.get()), minUnreadTimestampMsSinceEpoch.isSignificant(), - new Instant(lastReceivedMsSinceEpoch)); + Instant.ofEpochMilli(lastReceivedMsSinceEpoch)); lastLogTimestampMsSinceEpoch = nowMsSinceEpoch; } diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/EFOKinesisReaderTest.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/EFOKinesisReaderTest.java index dc6dc31f1a3b..9f226066385b 100644 --- a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/EFOKinesisReaderTest.java +++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/EFOKinesisReaderTest.java @@ -135,7 +135,7 @@ public void readsThroughAllDataAvailable() throws IOException { @Test public void returnsCurrentWatermark() throws IOException { - Instant expectedWatermark = new Instant(123456L); + Instant expectedWatermark = Instant.ofEpochMilli(123456L); when(subscribersPool.getWatermark()).thenReturn(expectedWatermark); reader.start(); diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisReaderTest.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisReaderTest.java index c063e817244c..60d102e84ec8 100644 --- a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisReaderTest.java +++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisReaderTest.java @@ -126,7 +126,7 @@ public void readsThroughAllDataAvailable() throws IOException { @Test public void returnsCurrentWatermark() throws IOException { - Instant expectedWatermark = new Instant(123456L); + Instant expectedWatermark = Instant.ofEpochMilli(123456L); when(shardReadersPool.getWatermark()).thenReturn(expectedWatermark); reader.start(); diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/RecordsAggregatorTest.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/RecordsAggregatorTest.java index c9a317f31112..113593a782b4 100644 --- a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/RecordsAggregatorTest.java +++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/RecordsAggregatorTest.java @@ -44,7 +44,7 @@ public class RecordsAggregatorTest { private static final String HASH_KEY = "12345"; private static final int HASH_KEY_OVERHEAD = 2 + HASH_KEY.length(); - private RecordsAggregator aggregator = new RecordsAggregator(MAX_BYTES_PER_RECORD, new Instant()); + private RecordsAggregator aggregator = new RecordsAggregator(MAX_BYTES_PER_RECORD, Instant.now()); @Test public void testAggregationCompatibilityWithKcl() { @@ -129,7 +129,7 @@ public void testSizeIncrement() { @Test public void testRejectRecordIfSizeExceeded() { - aggregator = new RecordsAggregator(BASE_OVERHEAD + PARTITION_KEY_OVERHEAD + 100, new Instant()); + aggregator = new RecordsAggregator(BASE_OVERHEAD + PARTITION_KEY_OVERHEAD + 100, Instant.now()); // adding record fails due to encoding overhead assertThat(aggregator.addRecord(PARTITION_KEY, null, new byte[95])).isFalse(); // but can fit if size is reduced @@ -138,7 +138,7 @@ public void testRejectRecordIfSizeExceeded() { @Test public void testHasCapacity() { - aggregator = new RecordsAggregator(BASE_OVERHEAD + PARTITION_KEY_OVERHEAD + 100, new Instant()); + aggregator = new RecordsAggregator(BASE_OVERHEAD + PARTITION_KEY_OVERHEAD + 100, Instant.now()); assertThat(aggregator.addRecord(PARTITION_KEY, null, new byte[30])).isTrue(); assertThat(aggregator.hasCapacity()).isTrue(); // can fit next record of avg size assertThat(aggregator.addRecord(PARTITION_KEY, null, new byte[30])).isTrue(); diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/TestHelpers.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/TestHelpers.java index 485f2def20e2..42aa864f55af 100644 --- a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/TestHelpers.java +++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/TestHelpers.java @@ -102,7 +102,7 @@ static List> createAggregatedRecords(int shards, int events) { final Instant now = DateTime.now().toInstant(); Function> dataStream = shard -> { - RecordsAggregator aggregator = new RecordsAggregator(1024, new org.joda.time.Instant()); + RecordsAggregator aggregator = new RecordsAggregator(1024, org.joda.time.Instant.now()); List records = range(0, events).mapToObj(off -> record(now, shard, off)).collect(toList()); for (Record record : records) { @@ -256,7 +256,7 @@ static SubscribeToShardEvent eventWithRecords(int startSeqNumber, int numRecords } static SubscribeToShardEvent eventWithAggRecords(int startSeqNumber, int numRecords) { - RecordsAggregator aggregator = new RecordsAggregator(1024, new org.joda.time.Instant()); + RecordsAggregator aggregator = new RecordsAggregator(1024, org.joda.time.Instant.now()); for (int i = startSeqNumber; i < startSeqNumber + numRecords; i++) { aggregator.addRecord("foo", null, String.valueOf(i).getBytes(UTF_8)); } diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/s3/testing/S3FileSystemIT.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/s3/testing/S3FileSystemIT.java index e42f04d5959e..942fce731d41 100644 --- a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/s3/testing/S3FileSystemIT.java +++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/s3/testing/S3FileSystemIT.java @@ -91,7 +91,7 @@ public void testWriteThenRead() { } static class S3Bucket extends ExternalResource { - public final String name = "beam-s3io-it-" + new DateTime().toString("yyyyMMdd-HHmmss"); + public final String name = "beam-s3io-it-" + DateTime.now().toString("yyyyMMdd-HHmmss"); @Override protected void before() throws Throwable { diff --git a/sdks/java/io/amqp/src/main/java/org/apache/beam/sdk/io/amqp/AmqpIO.java b/sdks/java/io/amqp/src/main/java/org/apache/beam/sdk/io/amqp/AmqpIO.java index 696332abf05e..dd1bd661bf66 100644 --- a/sdks/java/io/amqp/src/main/java/org/apache/beam/sdk/io/amqp/AmqpIO.java +++ b/sdks/java/io/amqp/src/main/java/org/apache/beam/sdk/io/amqp/AmqpIO.java @@ -238,7 +238,7 @@ private static class UnboundedAmqpReader extends UnboundedSource.UnboundedReader private Messenger messenger; private Message current; private Instant currentTimestamp; - private Instant watermark = new Instant(Long.MIN_VALUE); + private Instant watermark = Instant.ofEpochMilli(Long.MIN_VALUE); private AmqpCheckpointMark checkpointMark; public UnboundedAmqpReader(UnboundedAmqpSource source, AmqpCheckpointMark checkpointMark) { @@ -304,7 +304,7 @@ public boolean advance() { Message message = messenger.get(); Tracker tracker = messenger.incomingTracker(); checkpointMark.trackers.add(tracker); - currentTimestamp = new Instant(message.getCreationTime()); + currentTimestamp = Instant.ofEpochMilli(message.getCreationTime()); watermark = currentTimestamp; current = message; return true; diff --git a/sdks/java/io/clickhouse/src/main/java/org/apache/beam/sdk/io/clickhouse/ClickHouseWriter.java b/sdks/java/io/clickhouse/src/main/java/org/apache/beam/sdk/io/clickhouse/ClickHouseWriter.java index 73735f568646..d37a7c6a957f 100644 --- a/sdks/java/io/clickhouse/src/main/java/org/apache/beam/sdk/io/clickhouse/ClickHouseWriter.java +++ b/sdks/java/io/clickhouse/src/main/java/org/apache/beam/sdk/io/clickhouse/ClickHouseWriter.java @@ -37,7 +37,7 @@ "nullness" // TODO(https://github.com/apache/beam/issues/20497) }) public class ClickHouseWriter { - private static final Instant EPOCH_INSTANT = new Instant(0L); + private static final Instant EPOCH_INSTANT = Instant.ofEpochMilli(0L); @SuppressWarnings("unchecked") static void writeNullableValue(ClickHouseOutputStream stream, ColumnType columnType, Object value) diff --git a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java index 2c911b2014c3..c6ad2afde207 100644 --- a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java +++ b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java @@ -1383,10 +1383,11 @@ void testWriteWindowPreservation() throws IOException { Duration stepDuration = Duration.standardSeconds(step); Duration offset = Duration.ZERO; TestStream.Builder docsBuilder = - TestStream.create(StringUtf8Coder.of()).advanceWatermarkTo(new Instant(0)); + TestStream.create(StringUtf8Coder.of()).advanceWatermarkTo(Instant.ofEpochMilli(0)); for (String doc : data) { - docsBuilder = docsBuilder.addElements(TimestampedValue.of(doc, new Instant(0).plus(offset))); + docsBuilder = + docsBuilder.addElements(TimestampedValue.of(doc, Instant.ofEpochMilli(0).plus(offset))); offset = offset.plus(stepDuration); } @@ -1412,8 +1413,8 @@ void testWriteWindowPreservation() throws IOException { PAssert.that(successfulWrites) .inWindow( new IntervalWindow( - new Instant(0).plus(windowDuration.multipliedBy(i)), - new Instant(0).plus(windowDuration.multipliedBy(i + 1)))) + Instant.ofEpochMilli(0).plus(windowDuration.multipliedBy(i)), + Instant.ofEpochMilli(0).plus(windowDuration.multipliedBy(i + 1)))) .satisfies( windowPreservationValidator(windowSize, i * windowSize, (i + 1) * windowSize - 1)); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java index 16dbc2b5f186..6668eb97653c 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java @@ -1087,7 +1087,7 @@ private static ReadableInstant safeToMillis(Object value) { } private static ReadableInstant truncateToMillis(Object value) { - return new Instant((long) value / 1000); + return Instant.ofEpochMilli((long) value / 1000); } private static Object convertAvroArray( diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java index b9a554d54ade..e923211b373a 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java @@ -820,7 +820,7 @@ private void stats() { messageSkew, watermarkSkew, numLateMessages.get(nowMsSinceEpoch), - new Instant(lastWatermarkMsSinceEpoch)); + Instant.ofEpochMilli(lastWatermarkMsSinceEpoch)); lastLogTimestampMsSinceEpoch = nowMsSinceEpoch; } @@ -904,7 +904,7 @@ public Instant getCurrentTimestamp() throws NoSuchElementException { if (current == null) { throw new NoSuchElementException(); } - return new Instant(current.timestampMsSinceEpoch()); + return Instant.ofEpochMilli(current.timestampMsSinceEpoch()); } @Override @@ -982,7 +982,7 @@ public Instant getWatermark() { // else: We're not confident enough to estimate a new watermark. Stick with the old one. minWatermarkMsSinceEpoch.add(nowMsSinceEpoch, lastWatermarkMsSinceEpoch); maxWatermarkMsSinceEpoch.add(nowMsSinceEpoch, lastWatermarkMsSinceEpoch); - return new Instant(lastWatermarkMsSinceEpoch); + return Instant.ofEpochMilli(lastWatermarkMsSinceEpoch); } @Override diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/TestPubsub.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/TestPubsub.java index 2736424e8dc0..7046dac57de9 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/TestPubsub.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/TestPubsub.java @@ -333,10 +333,10 @@ public List waitForNMessages(int n, Duration timeoutDuration) .build(); subscriber.startAsync(); - DateTime startTime = new DateTime(); + DateTime startTime = DateTime.now(); int timeoutSeconds = timeoutDuration.toStandardSeconds().getSeconds(); while (receivedMessages.size() < n - && Seconds.secondsBetween(startTime, new DateTime()).getSeconds() < timeoutSeconds) { + && Seconds.secondsBetween(startTime, DateTime.now()).getSeconds() < timeoutSeconds) { try { Thread.sleep(1000); } catch (InterruptedException ignored) { @@ -411,10 +411,10 @@ public void assertSubscriptionEventuallyCreated(String project, Duration timeout throw new IllegalArgumentException(String.format("timeoutDuration should be greater than 0")); } - DateTime startTime = new DateTime(); + DateTime startTime = DateTime.now(); int sizeOfSubscriptionList = 0; while (sizeOfSubscriptionList == 0 - && Seconds.secondsBetween(startTime, new DateTime()).getSeconds() + && Seconds.secondsBetween(startTime, DateTime.now()).getSeconds() < timeoutDuration.toStandardSeconds().getSeconds()) { // Sleep 1 sec Thread.sleep(1000); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/TestPubsubSignal.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/TestPubsubSignal.java index 1d430ac4a6b6..6d054bb7bce1 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/TestPubsubSignal.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/TestPubsubSignal.java @@ -316,10 +316,10 @@ private String pollForResultForDuration( .build(); subscriber.startAsync(); - DateTime startTime = new DateTime(); + DateTime startTime = DateTime.now(); int timeoutSeconds = timeoutDuration.toStandardSeconds().getSeconds(); while (result.get() == null - && Seconds.secondsBetween(startTime, new DateTime()).getSeconds() < timeoutSeconds) { + && Seconds.secondsBetween(startTime, DateTime.now()).getSeconds() < timeoutSeconds) { try { Thread.sleep(1000); } catch (InterruptedException ignored) { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/ChildPartitionsRecordAction.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/ChildPartitionsRecordAction.java index 14b6b2e2453a..e1c324d85f48 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/ChildPartitionsRecordAction.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/ChildPartitionsRecordAction.java @@ -117,7 +117,7 @@ public Optional run( LOG.debug("[{}] Processing child partition record {}", token, record); final Timestamp startTimestamp = record.getStartTimestamp(); - final Instant startInstant = new Instant(startTimestamp.toSqlTimestamp().getTime()); + final Instant startInstant = Instant.ofEpochMilli(startTimestamp.toSqlTimestamp().getTime()); if (interrupter.tryInterrupt(startTimestamp)) { LOG.debug( "[{}] Soft deadline reached with child partitions record at {}, rescheduling", diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/DataChangeRecordAction.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/DataChangeRecordAction.java index 555b1fefbebc..e27591f41aae 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/DataChangeRecordAction.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/DataChangeRecordAction.java @@ -91,7 +91,7 @@ public Optional run( LOG.debug("[{}] Processing data record {}", token, record.getCommitTimestamp()); final Timestamp commitTimestamp = record.getCommitTimestamp(); - final Instant commitInstant = new Instant(commitTimestamp.toSqlTimestamp().getTime()); + final Instant commitInstant = Instant.ofEpochMilli(commitTimestamp.toSqlTimestamp().getTime()); if (interrupter.tryInterrupt(commitTimestamp)) { LOG.debug( "[{}] Soft deadline reached with data change record at {}, rescheduling", diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/HeartbeatRecordAction.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/HeartbeatRecordAction.java index 0937e896fbf1..3af773ddf427 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/HeartbeatRecordAction.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/HeartbeatRecordAction.java @@ -82,7 +82,7 @@ public Optional run( LOG.debug("[{}] Processing heartbeat record {}", token, record); final Timestamp timestamp = record.getTimestamp(); - final Instant timestampInstant = new Instant(timestamp.toSqlTimestamp().getTime()); + final Instant timestampInstant = Instant.ofEpochMilli(timestamp.toSqlTimestamp().getTime()); if (interrupter.tryInterrupt(timestamp)) { LOG.debug( "[{}] Soft deadline reached with heartbeat record at {}, rescheduling", token, timestamp); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/PartitionEndRecordAction.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/PartitionEndRecordAction.java index e59fb8028c00..06cbde51953f 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/PartitionEndRecordAction.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/PartitionEndRecordAction.java @@ -86,7 +86,7 @@ public Optional run( LOG.debug("[{}] Processing partition end record {}", token, record); final Timestamp timestamp = record.getEndTimestamp(); - final Instant timestampInstant = new Instant(timestamp.toSqlTimestamp().getTime()); + final Instant timestampInstant = Instant.ofEpochMilli(timestamp.toSqlTimestamp().getTime()); if (interrupter.tryInterrupt(timestamp)) { LOG.debug( "[{}] Soft deadline reached with partition end record at {}, rescheduling", diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/PartitionEventRecordAction.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/PartitionEventRecordAction.java index e99106c7e142..06dfa3da68f0 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/PartitionEventRecordAction.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/PartitionEventRecordAction.java @@ -86,7 +86,7 @@ public Optional run( LOG.debug("[{}] Processing partition event record {}", token, record); final Timestamp timestamp = record.getCommitTimestamp(); - final Instant timestampInstant = new Instant(timestamp.toSqlTimestamp().getTime()); + final Instant timestampInstant = Instant.ofEpochMilli(timestamp.toSqlTimestamp().getTime()); if (interrupter.tryInterrupt(timestamp)) { LOG.debug( "[{}] Soft deadline reached with partition event record at {}, rescheduling", diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/PartitionStartRecordAction.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/PartitionStartRecordAction.java index 3d20b858e4ac..62f8b4b12aad 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/PartitionStartRecordAction.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/PartitionStartRecordAction.java @@ -117,7 +117,8 @@ public Optional run( LOG.debug("[{}] Could not claim queryChangeStream({}), stopping", token, startTimestamp); return Optional.of(ProcessContinuation.stop()); } - watermarkEstimator.setWatermark(new Instant(startTimestamp.toSqlTimestamp().getTime())); + watermarkEstimator.setWatermark( + Instant.ofEpochMilli(startTimestamp.toSqlTimestamp().getTime())); for (String startPartitionToken : record.getPartitionTokens()) { processStartPartition(partition, record, startPartitionToken); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dofn/PostProcessingMetricsDoFn.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dofn/PostProcessingMetricsDoFn.java index 3529ba336e37..518610c62191 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dofn/PostProcessingMetricsDoFn.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/dofn/PostProcessingMetricsDoFn.java @@ -72,7 +72,7 @@ public PostProcessingMetricsDoFn(ChangeStreamMetrics metrics) { public void processElement( @Element DataChangeRecord dataChangeRecord, OutputReceiver receiver) { final Instant commitInstant = - new Instant(dataChangeRecord.getCommitTimestamp().toSqlTimestamp().getTime()); + Instant.ofEpochMilli(dataChangeRecord.getCommitTimestamp().toSqlTimestamp().getTime()); metrics.incDataRecordCounter(); measureCommitTimestampToEmittedMillis(dataChangeRecord); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java index a5d6ac68ce66..ff49036715cd 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java @@ -790,7 +790,7 @@ public void testStreamingFileLoadsWithAutoSharding() throws Exception { elements.add(new TableRow().set("number", i)); } - Instant startInstant = new Instant(0L); + Instant startInstant = Instant.ofEpochMilli(0L); TestStream testStream = TestStream.create(TableRowJsonCoder.of()) // Initialize watermark for timer to be triggered correctly. @@ -2460,7 +2460,7 @@ public void updateTableSchemaTest(boolean useSet) throws Exception { LongFunction getRow = useSet ? getRowSet : getRowSetF; TestStream.Builder testStream = - TestStream.create(VarLongCoder.of()).advanceWatermarkTo(new Instant(0)); + TestStream.create(VarLongCoder.of()).advanceWatermarkTo(Instant.ofEpochMilli(0)); // These rows contain unknown fields, which should be dropped. for (long i = 0; i < 5; i++) { testStream = testStream.addElements(i); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilsTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilsTest.java index b50e8448698a..0499e99ad1f8 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilsTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilsTest.java @@ -929,7 +929,7 @@ public void testMilliPrecisionOk() { long millis = 123456789L; assertThat( BigQueryUtils.convertAvroFormat(FieldType.DATETIME, millis * 1000, REJECT_OPTIONS), - equalTo(new Instant(millis))); + equalTo(Instant.ofEpochMilli(millis))); } @Test @@ -937,7 +937,7 @@ public void testSubMilliPrecisionTruncated() { long millis = 123456789L; assertThat( BigQueryUtils.convertAvroFormat(FieldType.DATETIME, millis * 1000 + 123, TRUNCATE_OPTIONS), - equalTo(new Instant(millis))); + equalTo(Instant.ofEpochMilli(millis))); } @Test diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FileLoadsStreamingIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FileLoadsStreamingIT.java index 678708062b8d..a095002f9782 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FileLoadsStreamingIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FileLoadsStreamingIT.java @@ -240,7 +240,7 @@ private void runStreaming(int numFileShards, boolean useCopyJobs) String tableSpec = maybeCreateTable(bqTableSchema, ""); // set up and build pipeline - Instant start = new Instant(0); + Instant start = Instant.ofEpochMilli(0); GenerateRowFunc generateRowFunc = new GenerateRowFunc(fieldNamesShuffled); PCollection instants = p.apply( @@ -372,7 +372,7 @@ private void runStreamingToDynamicDestinations(int numFileShards, boolean useCop String tablePrefix = table0Id.substring(0, table0Id.length() - 2); // set up and build pipeline - Instant start = new Instant(0); + Instant start = Instant.ofEpochMilli(0); PCollection instants = p.apply( "Generate Instants", diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiSinkSchemaUpdateIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiSinkSchemaUpdateIT.java index 3cb9897ada52..22866b2f8ec3 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiSinkSchemaUpdateIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiSinkSchemaUpdateIT.java @@ -421,7 +421,7 @@ private void runStreamingPipelineWithSchemaChange( } // set up and build pipeline - Instant start = new Instant(0); + Instant start = Instant.ofEpochMilli(0); Duration interval = waitLonger ? Duration.standardSeconds(LONG_WAIT_SECONDS) : Duration.millis(1); Duration stop = @@ -683,7 +683,7 @@ public void runDynamicDestinationsWithAutoSchemaUpdate(boolean useAtLeastOnce) t int numRows = TOTAL_N; // set up and build pipeline - Instant start = new Instant(0); + Instant start = Instant.ofEpochMilli(0); // We give a healthy waiting period between each element to give Storage API streams a chance to // recognize the new schema. Apply on relevant tests. Duration interval = diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/providers/BigQueryManagedIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/providers/BigQueryManagedIT.java index 4c164e6a38db..5affdaab0dcd 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/providers/BigQueryManagedIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/providers/BigQueryManagedIT.java @@ -219,8 +219,8 @@ public PCollection getInput(Pipeline p, boolean isStreaming) { if (isStreaming) { return p.apply( PeriodicImpulse.create() - .startAt(new Instant(0)) - .stopAt(new Instant(19)) + .startAt(Instant.ofEpochMilli(0)) + .stopAt(Instant.ofEpochMilli(19)) .withInterval(Duration.millis(1))) .apply(MapElements.into(TypeDescriptors.rows()).via(i -> ROW_FUNC.apply(i.getMillis()))) .setRowSchema(SCHEMA); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOTest.java index 3d9c65aa1376..6cbd6922e909 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOTest.java @@ -636,9 +636,9 @@ public void testAvroPojo() { List inputs = Lists.newArrayList( new GenericClass( - 1, "foo", new DateTime().withDate(2019, 10, 1).withZone(DateTimeZone.UTC)), + 1, "foo", DateTime.now().withDate(2019, 10, 1).withZone(DateTimeZone.UTC)), new GenericClass( - 2, "bar", new DateTime().withDate(1986, 10, 1).withZone(DateTimeZone.UTC))); + 2, "bar", DateTime.now().withDate(1986, 10, 1).withZone(DateTimeZone.UTC))); setupTestClient(inputs, coder); PCollection read = pipeline.apply( diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSinkTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSinkTest.java index 49b7f88f9e25..3b2e5f26c569 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSinkTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSinkTest.java @@ -94,7 +94,7 @@ public void processElement(ProcessContext c) { c.outputWithTimestamp( new PubsubMessage(c.element().getBytes(StandardCharsets.UTF_8), attributes) .withOrderingKey(orderingKeyFn.apply(c.element())), - new Instant(TIMESTAMP)); + Instant.ofEpochMilli(TIMESTAMP)); } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSourceTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSourceTest.java index d3087df92386..775d4f2c8fb4 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSourceTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSourceTest.java @@ -364,7 +364,8 @@ public void readManyMessages() throws Exception { // No duplicate messages. assertNotNull(messageNum); // Preserve timestamp. - assertEquals(new Instant(messageNumToTimestamp(messageNum)), reader.getCurrentTimestamp()); + assertEquals( + Instant.ofEpochMilli(messageNumToTimestamp(messageNum)), reader.getCurrentTimestamp()); // Preserve record id. String recid = String.format("recordid_%d", messageNum); assertArrayEquals(recid.getBytes(StandardCharsets.UTF_8), reader.getCurrentRecordId()); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/ChildPartitionsRecordActionTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/ChildPartitionsRecordActionTest.java index 38e3b7fdfa0a..9f197768498f 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/ChildPartitionsRecordActionTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/ChildPartitionsRecordActionTest.java @@ -97,7 +97,8 @@ public void testRestrictionClaimedAndIsSplitCase() { action.run(partition, record, tracker, interrupter, watermarkEstimator); assertEquals(Optional.empty(), maybeContinuation); - verify(watermarkEstimator).setWatermark(new Instant(startTimestamp.toSqlTimestamp().getTime())); + verify(watermarkEstimator) + .setWatermark(Instant.ofEpochMilli(startTimestamp.toSqlTimestamp().getTime())); verify(transaction) .insert( PartitionMetadata.newBuilder() @@ -148,7 +149,8 @@ public void testRestrictionClaimedAnsIsSplitCaseAndChildExists() { action.run(partition, record, tracker, interrupter, watermarkEstimator); assertEquals(Optional.empty(), maybeContinuation); - verify(watermarkEstimator).setWatermark(new Instant(startTimestamp.toSqlTimestamp().getTime())); + verify(watermarkEstimator) + .setWatermark(Instant.ofEpochMilli(startTimestamp.toSqlTimestamp().getTime())); } @Test @@ -177,7 +179,8 @@ public void testRestrictionClaimedAndIsMergeCaseAndChildNotExists() { action.run(partition, record, tracker, interrupter, watermarkEstimator); assertEquals(Optional.empty(), maybeContinuation); - verify(watermarkEstimator).setWatermark(new Instant(startTimestamp.toSqlTimestamp().getTime())); + verify(watermarkEstimator) + .setWatermark(Instant.ofEpochMilli(startTimestamp.toSqlTimestamp().getTime())); verify(transaction) .insert( PartitionMetadata.newBuilder() @@ -217,7 +220,8 @@ public void testRestrictionClaimedAndIsMergeCaseAndChildExists() { action.run(partition, record, tracker, interrupter, watermarkEstimator); assertEquals(Optional.empty(), maybeContinuation); - verify(watermarkEstimator).setWatermark(new Instant(startTimestamp.toSqlTimestamp().getTime())); + verify(watermarkEstimator) + .setWatermark(Instant.ofEpochMilli(startTimestamp.toSqlTimestamp().getTime())); verify(transaction, never()).insert(any()); } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/DataChangeRecordActionTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/DataChangeRecordActionTest.java index 6569f810812c..7f364eee9000 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/DataChangeRecordActionTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/DataChangeRecordActionTest.java @@ -65,7 +65,7 @@ public void setUp() { public void testRestrictionClaimed() { final String partitionToken = "partitionToken"; final Timestamp timestamp = Timestamp.ofTimeMicroseconds(10L); - final Instant instant = new Instant(timestamp.toSqlTimestamp().getTime()); + final Instant instant = Instant.ofEpochMilli(timestamp.toSqlTimestamp().getTime()); final DataChangeRecord record = mock(DataChangeRecord.class); when(record.getCommitTimestamp()).thenReturn(timestamp); when(tracker.tryClaim(timestamp)).thenReturn(true); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/HeartbeatRecordActionTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/HeartbeatRecordActionTest.java index 56d1825c8a18..b785a8b7ac92 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/HeartbeatRecordActionTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/HeartbeatRecordActionTest.java @@ -73,7 +73,8 @@ public void testRestrictionClaimed() { watermarkEstimator); assertEquals(Optional.empty(), maybeContinuation); - verify(watermarkEstimator).setWatermark(new Instant(timestamp.toSqlTimestamp().getTime())); + verify(watermarkEstimator) + .setWatermark(Instant.ofEpochMilli(timestamp.toSqlTimestamp().getTime())); } @Test diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/PartitionEndRecordActionTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/PartitionEndRecordActionTest.java index c6bc24832ebb..d35ddcb3d305 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/PartitionEndRecordActionTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/PartitionEndRecordActionTest.java @@ -73,7 +73,8 @@ public void testRestrictionClaimed() { watermarkEstimator); assertEquals(Optional.empty(), maybeContinuation); - verify(watermarkEstimator).setWatermark(new Instant(timestamp.toSqlTimestamp().getTime())); + verify(watermarkEstimator) + .setWatermark(Instant.ofEpochMilli(timestamp.toSqlTimestamp().getTime())); } @Test diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/PartitionEventRecordActionTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/PartitionEventRecordActionTest.java index fdc3ff641287..4bf45bcc802e 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/PartitionEventRecordActionTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/PartitionEventRecordActionTest.java @@ -73,7 +73,8 @@ public void testRestrictionClaimed() { watermarkEstimator); assertEquals(Optional.empty(), maybeContinuation); - verify(watermarkEstimator).setWatermark(new Instant(timestamp.toSqlTimestamp().getTime())); + verify(watermarkEstimator) + .setWatermark(Instant.ofEpochMilli(timestamp.toSqlTimestamp().getTime())); } @Test diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/PartitionStartRecordActionTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/PartitionStartRecordActionTest.java index b37f3d6b198f..49a267579ff3 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/PartitionStartRecordActionTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/action/PartitionStartRecordActionTest.java @@ -86,7 +86,8 @@ public void testRestrictionClaimed() { action.run(partition, record, tracker, interrupter, watermarkEstimator); assertEquals(Optional.empty(), maybeContinuation); - verify(watermarkEstimator).setWatermark(new Instant(startTimestamp.toSqlTimestamp().getTime())); + verify(watermarkEstimator) + .setWatermark(Instant.ofEpochMilli(startTimestamp.toSqlTimestamp().getTime())); verify(transaction) .insert( PartitionMetadata.newBuilder() diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamTransactionBoundariesIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamTransactionBoundariesIT.java index 4ff4f9a81e73..77263a5db6b3 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamTransactionBoundariesIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/changestreams/it/SpannerChangeStreamTransactionBoundariesIT.java @@ -205,7 +205,8 @@ public void process( .collect(Collectors.toList()); final Instant commitInstant = - new Instant(sortedRecords.get(0).getCommitTimestamp().toSqlTimestamp().getTime()); + Instant.ofEpochMilli( + sortedRecords.get(0).getCommitTimestamp().toSqlTimestamp().getTime()); context.outputWithTimestamp( KV.of( new SpannerChangeStreamTransactionBoundariesIT.SortKey( diff --git a/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIOSequenceFileTest.java b/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIOSequenceFileTest.java index b6cb6e627b6f..9dfd5b4748fc 100644 --- a/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIOSequenceFileTest.java +++ b/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIOSequenceFileTest.java @@ -74,7 +74,7 @@ @RunWith(JUnit4.class) public class HadoopFormatIOSequenceFileTest { - private static final Instant START_TIME = new Instant(0); + private static final Instant START_TIME = Instant.ofEpochMilli(0); private static final String TEST_FOLDER_NAME = "test"; private static final String LOCKS_FOLDER_NAME = "locks"; private static final int REDUCERS_COUNT = 2; diff --git a/sdks/java/io/hcatalog/src/main/java/org/apache/beam/sdk/io/hcatalog/HCatToRow.java b/sdks/java/io/hcatalog/src/main/java/org/apache/beam/sdk/io/hcatalog/HCatToRow.java index e5bdf18ecbcf..e8ab014cb5b9 100644 --- a/sdks/java/io/hcatalog/src/main/java/org/apache/beam/sdk/io/hcatalog/HCatToRow.java +++ b/sdks/java/io/hcatalog/src/main/java/org/apache/beam/sdk/io/hcatalog/HCatToRow.java @@ -31,7 +31,8 @@ /** Utilities to convert {@link HCatRecord HCatRecords} to {@link Row Rows}. */ @SuppressWarnings({ - "nullness" // TODO(https://github.com/apache/beam/issues/20497) + "nullness", // TODO(https://github.com/apache/beam/issues/20497) + "JodaConstructors" }) public class HCatToRow { diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOWriteTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOWriteTest.java index a7349bffdfa0..352eadec6a26 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOWriteTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOWriteTest.java @@ -368,7 +368,7 @@ public void testStreamingWrite() { List inputRows = TestFixtures.asRows(TestFixtures.FILE1SNAPSHOT1); TestStream stream = TestStream.create(IcebergUtils.icebergSchemaToBeamSchema(TestFixtures.SCHEMA)) - .advanceWatermarkTo(new Instant(0)) + .advanceWatermarkTo(Instant.ofEpochMilli(0)) // the first two rows are written within the same triggering interval, // so they should both be in the first snapshot .addElements(inputRows.get(0)) diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergUtilsTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergUtilsTest.java index c9026522dba3..b2c45c5b055c 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergUtilsTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergUtilsTest.java @@ -139,7 +139,7 @@ public void testTimestamp() { // Schema.FieldType.DATETIME DateTime dateTime = - new DateTime().withDate(1979, 03, 14).withTime(1, 2, 3, 4).withZone(DateTimeZone.UTC); + DateTime.now().withDate(1979, 03, 14).withTime(1, 2, 3, 4).withZone(DateTimeZone.UTC); checkRowValueToRecordValue( Schema.FieldType.DATETIME, dateTime, @@ -348,7 +348,7 @@ public void testTimestamp() { // Schema.FieldType.DATETIME DateTime dateTime = - new DateTime().withDate(1979, 03, 14).withTime(1, 2, 3, 4).withZone(DateTimeZone.UTC); + DateTime.now().withDate(1979, 03, 14).withTime(1, 2, 3, 4).withZone(DateTimeZone.UTC); checkRecordValueToRowValue( Types.TimestampType.withoutZone(), dateTime.getMillis() * 1000L, diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProviderTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProviderTest.java index 7028a394d2fd..d86158c4b4a5 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProviderTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProviderTest.java @@ -255,7 +255,7 @@ private void writeToDynamicDestinationsAndFilter(@Nullable String operation, boo // use interpolator to fetch destinations identifiers. create iceberg tables beforehand RowStringInterpolator interpolator = new RowStringInterpolator(destinationTemplate, beamSchema); - Instant first = new Instant(0); + Instant first = Instant.ofEpochMilli(0); Instant second = first.plus(Duration.standardDays(1)); Instant third = second.plus(Duration.standardDays(1)); String identifier0 = diff --git a/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOTest.java b/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOTest.java index 383b787e726d..f1e85c117776 100644 --- a/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOTest.java +++ b/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOTest.java @@ -1003,7 +1003,7 @@ public void testGetPreparedStatementSetCaller() throws Exception { 20.5D, 15.5F, 10, - new DateTime(), + DateTime.now(), (short) 5, Byte.parseByte("1", 2)) .build(); diff --git a/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsCheckpointMark.java b/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsCheckpointMark.java index e213561917d6..0e21272e17c5 100644 --- a/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsCheckpointMark.java +++ b/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsCheckpointMark.java @@ -145,7 +145,7 @@ void add(Message message) throws Exception { String.format( "Attempting to add message %s to checkpoint that is discarded.", message)); } - Instant currentMessageTimestamp = new Instant(message.getJMSTimestamp()); + Instant currentMessageTimestamp = Instant.ofEpochMilli(message.getJMSTimestamp()); if (currentMessageTimestamp.isBefore(oldestMessageTimestamp)) { oldestMessageTimestamp = currentMessageTimestamp; } diff --git a/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java b/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java index 2a7cd62d33d2..8aa61568b96a 100644 --- a/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java +++ b/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java @@ -683,7 +683,7 @@ public boolean advance() throws IOException { } currentMessage = this.source.spec.getMessageMapper().mapMessage(message); - currentTimestamp = new Instant(message.getJMSTimestamp()); + currentTimestamp = Instant.ofEpochMilli(message.getJMSTimestamp()); String messageID = message.getJMSMessageID(); if (messageID != null) { diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java index 02d14b745fe8..6fb06ee3a538 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java @@ -3121,7 +3121,7 @@ public static SerializableFunction, Instant> useCreateT record.getPartition(), record.getOffset(), record.getTimestampType()); - return new Instant(record.getTimestamp()); + return Instant.ofEpochMilli(record.getTimestamp()); }; } @@ -3135,7 +3135,7 @@ public static SerializableFunction, Instant> useLogAppe record.getPartition(), record.getOffset(), record.getTimestampType()); - return new Instant(record.getTimestamp()); + return Instant.ofEpochMilli(record.getTimestamp()); }; } } diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaUnboundedReader.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaUnboundedReader.java index 866dfd487108..ffa96a1b437a 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaUnboundedReader.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaUnboundedReader.java @@ -548,7 +548,7 @@ Instant updateAndGetWatermark() { partition, tp); nextOffset = ckptMark.getNextOffset(); - prevWatermark = Optional.of(new Instant(ckptMark.getWatermarkMillis())); + prevWatermark = Optional.of(Instant.ofEpochMilli(ckptMark.getWatermarkMillis())); } states.add( diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/TimestampPolicyFactory.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/TimestampPolicyFactory.java index 53a9a1c0004b..0f0a64188a6f 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/TimestampPolicyFactory.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/TimestampPolicyFactory.java @@ -85,7 +85,7 @@ record -> { record.getPartition(), record.getOffset(), record.getTimestampType()); - return new Instant(record.getTimestamp()); + return Instant.ofEpochMilli(record.getTimestamp()); }; return (tp, previousWatermark) -> @@ -148,7 +148,7 @@ public LogAppendTimePolicy(Optional previousWatermark) { @Override public Instant getTimestampForRecord(PartitionContext context, KafkaRecord record) { if (record.getTimestampType().equals(KafkaTimestampType.LOG_APPEND_TIME)) { - currentWatermark = new Instant(record.getTimestamp()); + currentWatermark = Instant.ofEpochMilli(record.getTimestamp()); } else if (currentWatermark.equals(BoundedWindow.TIMESTAMP_MIN_VALUE)) { // This is the first record and it does not have LOG_APPEND_TIME. // Most likely the topic is not configured correctly. diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/serialization/InstantDeserializer.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/serialization/InstantDeserializer.java index efdd8a83963c..27abd8684e0a 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/serialization/InstantDeserializer.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/serialization/InstantDeserializer.java @@ -35,7 +35,7 @@ public void configure(Map configs, boolean isKey) {} @Override public Instant deserialize(String topic, byte[] bytes) { - return new Instant(LONG_DESERIALIZER.deserialize(topic, bytes)); + return Instant.ofEpochMilli(LONG_DESERIALIZER.deserialize(topic, bytes)); } @Override diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/CustomTimestampPolicyWithLimitedDelayTest.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/CustomTimestampPolicyWithLimitedDelayTest.java index 59098fb3ec0e..5f834d8accca 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/CustomTimestampPolicyWithLimitedDelayTest.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/CustomTimestampPolicyWithLimitedDelayTest.java @@ -70,7 +70,7 @@ public void testCustomTimestampPolicyWithLimitedDelay() { CustomTimestampPolicyWithLimitedDelay policy = new CustomTimestampPolicyWithLimitedDelay<>( - record -> new Instant(record.getTimestamp()), maxDelay, Optional.empty()); + record -> Instant.ofEpochMilli(record.getTimestamp()), maxDelay, Optional.empty()); Instant now = Instant.now(); diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java index 703d323090dd..fc1b1cb3c1b4 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java @@ -195,7 +195,7 @@ public class KafkaIOTest { @Rule public ExpectedLogs kafkaIOExpectedLogs = ExpectedLogs.none(KafkaIO.class); - private static final Instant LOG_APPEND_START_TIME = new Instant(600 * 1000); + private static final Instant LOG_APPEND_START_TIME = Instant.ofEpochMilli(600 * 1000); private static final String TIMESTAMP_START_MILLIS_CONFIG = "test.timestamp.start.millis"; private static final String TIMESTAMP_TYPE_CONFIG = "test.timestamp.type"; static List mkKafkaTopics = ImmutableList.of("topic_a", "topic_b"); @@ -1210,7 +1210,7 @@ public void testUnboundedSourceCustomTimestamps() { (tp, prevWatermark) -> new CustomTimestampPolicyWithLimitedDelay( record -> - new Instant( + Instant.ofEpochMilli( TimeUnit.SECONDS.toMillis(record.getKV().getValue()) + customTimestampStartMillis), Duration.ZERO, @@ -1288,7 +1288,7 @@ public TimestampPolicy createTimestampPolicy( @Override public Instant getTimestampForRecord(PartitionContext ctx, KafkaRecord record) { lastOffset = record.getOffset(); - lastTimestamp = new Instant(record.getTimestamp()); + lastTimestamp = Instant.ofEpochMilli(record.getTimestamp()); return lastTimestamp; } @@ -1436,7 +1436,7 @@ public void testUnboundedSourceSplits() throws Exception { static class ValueAsTimestampFn implements SerializableFunction, Instant> { @Override public Instant apply(KV input) { - return new Instant(input.getValue()); + return Instant.ofEpochMilli(input.getValue()); } } @@ -2234,7 +2234,7 @@ public void testUnboundedSourceStartReadTime() { null, /*offsetDeduplication*/ null, /*topics*/ null /*redistributeByRecordKey*/) - .withStartReadTime(new Instant(startTime)) + .withStartReadTime(Instant.ofEpochMilli(startTime)) .withoutMetadata()) .apply(Values.create()); @@ -2311,7 +2311,7 @@ public void testUnboundedSourceStartReadTimeException() { null, /*offsetDeduplication*/ null, /*topics*/ null /*redistributeByRecordKey*/) - .withStartReadTime(new Instant(startTime)) + .withStartReadTime(Instant.ofEpochMilli(startTime)) .withoutMetadata()) .apply(Values.create()); diff --git a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbGridFSIO.java b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbGridFSIO.java index 71f8b291e0d5..5c7419de0a28 100644 --- a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbGridFSIO.java +++ b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbGridFSIO.java @@ -141,7 +141,7 @@ void parse( */ private static final Parser TEXT_PARSER = (gridFSFile, downloadStream, callback) -> { - final Instant time = new Instant(gridFSFile.getUploadDate().getTime()); + final Instant time = Instant.ofEpochMilli(gridFSFile.getUploadDate().getTime()); try (BufferedReader reader = new BufferedReader(new InputStreamReader(downloadStream, StandardCharsets.UTF_8))) { for (String line = reader.readLine(); line != null; line = reader.readLine()) { @@ -531,7 +531,7 @@ public Instant getCurrentTimestamp() throws NoSuchElementException { } long time = current.getTimestamp(); time *= 1000L; - return new Instant(time); + return Instant.ofEpochMilli(time); } @Override diff --git a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDBGridFSIOTest.java b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDBGridFSIOTest.java index d13185a08fb6..b933431e266b 100644 --- a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDBGridFSIOTest.java +++ b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDBGridFSIOTest.java @@ -220,7 +220,7 @@ public void testReadWithParser() { long timestamp = scanner.nextLong(); String name = scanner.next(); int score = scanner.nextInt(); - callback.output(KV.of(name, score), new Instant(timestamp)); + callback.output(KV.of(name, score), Instant.ofEpochMilli(timestamp)); } line = reader.readLine(); } diff --git a/sdks/java/io/pulsar/src/main/java/org/apache/beam/sdk/io/pulsar/NaiveReadFromPulsarDoFn.java b/sdks/java/io/pulsar/src/main/java/org/apache/beam/sdk/io/pulsar/NaiveReadFromPulsarDoFn.java index a80f02590827..4ccb7a760405 100644 --- a/sdks/java/io/pulsar/src/main/java/org/apache/beam/sdk/io/pulsar/NaiveReadFromPulsarDoFn.java +++ b/sdks/java/io/pulsar/src/main/java/org/apache/beam/sdk/io/pulsar/NaiveReadFromPulsarDoFn.java @@ -74,7 +74,7 @@ public class NaiveReadFromPulsarDoFn extends DoFn public NaiveReadFromPulsarDoFn(PulsarIO.Read transform) { this.extractOutputTimestampFn = transform.getTimestampType() == PulsarIO.ReadTimestampType.PUBLISH_TIME - ? record -> new Instant(record.getPublishTime()) + ? record -> Instant.ofEpochMilli(record.getPublishTime()) : ignored -> Instant.now(); this.pollingTimeout = Duration.ofSeconds(transform.getConsumerPollingTimeout()); this.outputFn = transform.getOutputFn(); diff --git a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/write/StreamingWriteTest.java b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/write/StreamingWriteTest.java index 780aa26356a7..65e62245baee 100644 --- a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/write/StreamingWriteTest.java +++ b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/write/StreamingWriteTest.java @@ -64,7 +64,7 @@ public class StreamingWriteTest { private static final String STAGING_BUCKET_NAME = "BUCKET/"; private static final String STORAGE_INTEGRATION_NAME = "STORAGE_INTEGRATION"; private static final String SNOW_PIPE = "Snowpipe"; - private static final Instant START_TIME = new Instant(0); + private static final Instant START_TIME = Instant.ofEpochMilli(0); @Rule public final transient TestPipeline pipeline = TestPipeline.create(); diff --git a/sdks/java/io/synthetic/src/main/java/org/apache/beam/sdk/io/synthetic/SyntheticUnboundedSource.java b/sdks/java/io/synthetic/src/main/java/org/apache/beam/sdk/io/synthetic/SyntheticUnboundedSource.java index 765b07cb85c2..ad09936a91ed 100644 --- a/sdks/java/io/synthetic/src/main/java/org/apache/beam/sdk/io/synthetic/SyntheticUnboundedSource.java +++ b/sdks/java/io/synthetic/src/main/java/org/apache/beam/sdk/io/synthetic/SyntheticUnboundedSource.java @@ -184,7 +184,7 @@ public boolean start() { public boolean advance() { currentOffset++; - processingTime = new Instant(); + processingTime = Instant.now(); eventTime = processingTime.minus(sourceOptions.nextProcessingTimeDelay(currentOffset)); SyntheticSourceOptions.Record record = diff --git a/sdks/java/io/synthetic/src/main/java/org/apache/beam/sdk/io/synthetic/SyntheticWatermark.java b/sdks/java/io/synthetic/src/main/java/org/apache/beam/sdk/io/synthetic/SyntheticWatermark.java index 1a900085bc3f..a628dee5ab88 100644 --- a/sdks/java/io/synthetic/src/main/java/org/apache/beam/sdk/io/synthetic/SyntheticWatermark.java +++ b/sdks/java/io/synthetic/src/main/java/org/apache/beam/sdk/io/synthetic/SyntheticWatermark.java @@ -40,7 +40,7 @@ class SyntheticWatermark implements Serializable { SyntheticWatermark(SyntheticSourceOptions options, long endOffset) { this.options = options; this.endOffset = endOffset; - this.watermark = new Instant(0); + this.watermark = Instant.ofEpochMilli(0); } /** Calculates new watermark value and returns it if it's greater than the previous one. */ diff --git a/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/SideInputLoadTest.java b/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/SideInputLoadTest.java index a0f219e20bcc..9aa65e938c37 100644 --- a/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/SideInputLoadTest.java +++ b/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/SideInputLoadTest.java @@ -62,7 +62,7 @@ public class SideInputLoadTest extends LoadTest { private static final String METRICS_NAMESPACE = "sideinput"; - private static final Instant TIME = new Instant(); + private static final Instant TIME = Instant.now(); public SideInputLoadTest(String[] args) throws IOException { super(args, Options.class, METRICS_NAMESPACE); diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkUtils.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkUtils.java index 2a536c4c6c7c..5f2a90041110 100644 --- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkUtils.java +++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkUtils.java @@ -354,7 +354,8 @@ public static void console(@FormatString String format, Object... args) { * Instants guaranteed to be strictly before and after all event timestamps, and which won't be * subject to underflow/overflow. */ - public static final Instant BEGINNING_OF_TIME = new Instant(0).plus(Duration.standardDays(365)); + public static final Instant BEGINNING_OF_TIME = + Instant.ofEpochMilli(0).plus(Duration.standardDays(365)); public static final Instant END_OF_TIME = BoundedWindow.TIMESTAMP_MAX_VALUE.minus(Duration.standardDays(365)); diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/NexmarkQueryModel.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/NexmarkQueryModel.java index 2144a1cc26c9..215db2ff92d8 100644 --- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/NexmarkQueryModel.java +++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/NexmarkQueryModel.java @@ -57,7 +57,7 @@ static Instant windowStart(Duration size, Duration period, Instant timestamp) { long p = period.getMillis(); long lim = ts - ts % p; long s = size.getMillis(); - return new Instant(lim - s); + return Instant.ofEpochMilli(lim - s); } /** Convert {@code itr} to strings capturing values and timestamps. */ diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/UnboundedEventSource.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/UnboundedEventSource.java index 1ec200555b69..d6c7ce78cf20 100644 --- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/UnboundedEventSource.java +++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/UnboundedEventSource.java @@ -181,7 +181,7 @@ public boolean advance() { } pendingEventWallclockTime = next.wallclockTimestamp; - pendingEvent = TimestampedValue.of(next.event, new Instant(next.eventTimestamp)); + pendingEvent = TimestampedValue.of(next.event, Instant.ofEpochMilli(next.eventTimestamp)); long newWatermark = next.watermark - Duration.standardSeconds(watermarkHoldbackSec).getMillis(); if (newWatermark > watermark) { @@ -272,7 +272,7 @@ public UnboundedEventSource getCurrentSource() { @Override public Instant getWatermark() { - return new Instant(watermark); + return Instant.ofEpochMilli(watermark); } @Override diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/generator/Generator.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/generator/Generator.java index 4174b587eea1..e6812485e9dc 100644 --- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/generator/Generator.java +++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/generator/Generator.java @@ -240,7 +240,7 @@ public NextEvent nextEvent() { @Override public TimestampedValue next() { NextEvent next = nextEvent(); - return TimestampedValue.of(next.event, new Instant(next.eventTimestamp)); + return TimestampedValue.of(next.event, Instant.ofEpochMilli(next.eventTimestamp)); } @Override diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/generator/model/AuctionGenerator.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/generator/model/AuctionGenerator.java index d92831877d5e..8c258211e6bb 100644 --- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/generator/model/AuctionGenerator.java +++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/generator/model/AuctionGenerator.java @@ -76,8 +76,8 @@ public static Auction nextAuction( desc, initialBid, reserve, - new Instant(timestamp), - new Instant(expires), + Instant.ofEpochMilli(timestamp), + Instant.ofEpochMilli(expires), seller, category, extra); diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/generator/model/BidGenerator.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/generator/model/BidGenerator.java index d5bd5d824865..58691addc6bd 100644 --- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/generator/model/BidGenerator.java +++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/generator/model/BidGenerator.java @@ -66,6 +66,6 @@ public static Bid nextBid(long eventId, Random random, long timestamp, Generator long price = PriceGenerator.nextPrice(random); int currentSize = 8 + 8 + 8 + 8; String extra = nextExtra(random, currentSize, config.getAvgBidByteSize()); - return new Bid(auction, bidder, price, new Instant(timestamp), extra); + return new Bid(auction, bidder, price, Instant.ofEpochMilli(timestamp), extra); } } diff --git a/sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery0Test.java b/sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery0Test.java index 51bd7ba00381..e8ddacb10660 100644 --- a/sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery0Test.java +++ b/sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery0Test.java @@ -36,9 +36,11 @@ @RunWith(Enclosed.class) public class SqlQuery0Test { - private static final Bid BID1 = new Bid(5L, 3L, 123123L, new Instant(43234234L), "extra1"); + private static final Bid BID1 = + new Bid(5L, 3L, 123123L, Instant.ofEpochMilli(43234234L), "extra1"); - private static final Bid BID2 = new Bid(6L, 4L, 134123L, new Instant(13234234L), "extra2"); + private static final Bid BID2 = + new Bid(6L, 4L, 134123L, Instant.ofEpochMilli(13234234L), "extra2"); private abstract static class SqlQuery0TestCases { protected abstract SqlQuery0 getQuery(); diff --git a/sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery1Test.java b/sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery1Test.java index 47723fef9de4..f9241862fd8d 100644 --- a/sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery1Test.java +++ b/sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery1Test.java @@ -34,13 +34,17 @@ /** Unit tests for {@link SqlQuery1}. */ public class SqlQuery1Test { - private static final Bid BID1_USD = new Bid(5L, 3L, 100L, new Instant(43234234L), "extra1"); + private static final Bid BID1_USD = + new Bid(5L, 3L, 100L, Instant.ofEpochMilli(43234234L), "extra1"); - private static final Bid BID2_USD = new Bid(6L, 4L, 500L, new Instant(13234234L), "extra2"); + private static final Bid BID2_USD = + new Bid(6L, 4L, 500L, Instant.ofEpochMilli(13234234L), "extra2"); - private static final Bid BID1_EUR = new Bid(5L, 3L, 89L, new Instant(43234234L), "extra1"); + private static final Bid BID1_EUR = + new Bid(5L, 3L, 89L, Instant.ofEpochMilli(43234234L), "extra1"); - private static final Bid BID2_EUR = new Bid(6L, 4L, 445L, new Instant(13234234L), "extra2"); + private static final Bid BID2_EUR = + new Bid(6L, 4L, 445L, Instant.ofEpochMilli(13234234L), "extra2"); @Rule public TestPipeline testPipeline = TestPipeline.create(); diff --git a/sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery2Test.java b/sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery2Test.java index 91b913521d71..cda082bf9424 100644 --- a/sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery2Test.java +++ b/sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery2Test.java @@ -98,7 +98,7 @@ protected SqlQuery2 getQuery(long skipFactor) { } private static Bid newBid(long id) { - return new Bid(id, 3L, 100L, new Instant(432342L + id), "extra_" + id); + return new Bid(id, 3L, 100L, Instant.ofEpochMilli(432342L + id), "extra_" + id); } private static AuctionPrice newAuctionPrice(Bid bid) { diff --git a/sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery3Test.java b/sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery3Test.java index 9c72900587cf..904fb77aee1e 100644 --- a/sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery3Test.java +++ b/sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery3Test.java @@ -115,7 +115,7 @@ private static Person newPerson(long id, String state) { "cc_" + id, "city_" + id, state, - new Instant(123123L + id), + Instant.ofEpochMilli(123123L + id), "extra_" + id); } @@ -126,8 +126,8 @@ private static Auction newAuction(long id, long seller, long category) { "desc_" + id, 123 + id, 200 + id, - new Instant(123123L + id), - new Instant(223123 + id), + Instant.ofEpochMilli(123123L + id), + Instant.ofEpochMilli(223123 + id), seller, category, "extra_" + id); diff --git a/sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery5Test.java b/sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery5Test.java index e040525613f8..26377f5d34c1 100644 --- a/sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery5Test.java +++ b/sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery5Test.java @@ -77,7 +77,7 @@ private static Bid newBid(long auction, long index) { auction, 3L, 100L, - new Instant(432342L + index * config.windowPeriodSec * 1000), + Instant.ofEpochMilli(432342L + index * config.windowPeriodSec * 1000), "extra_" + auction); } } diff --git a/sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery7Test.java b/sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery7Test.java index b1a392489719..1a18b0ef3f1e 100644 --- a/sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery7Test.java +++ b/sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery7Test.java @@ -70,7 +70,7 @@ private static Bid newBid(long auction, long bidder, long price, long index) { auction, bidder, price, - new Instant(432342L + index * config.windowSizeSec * 1000), + Instant.ofEpochMilli(432342L + index * config.windowSizeSec * 1000), "extra_" + auction); } } diff --git a/website/www/site/content/en/blog/test-stream.md b/website/www/site/content/en/blog/test-stream.md index 845dcb892ed8..7d079f64b9bd 100644 --- a/website/www/site/content/en/blog/test-stream.md +++ b/website/www/site/content/en/blog/test-stream.md @@ -127,11 +127,11 @@ PTransform: {{< highlight java >}} TestStream createEvents = TestStream.create(AvroCoder.of(GameActionInfo.class)) - .addElements(new GameActionInfo("sky", "blue", 12, new Instant(0L)), -                 new GameActionInfo("navy", "blue", 3, new Instant(0L)), -                 new GameActionInfo("navy", "blue", 3, new Instant(0L).plus(Duration.standardMinutes(3)))) + .addElements(new GameActionInfo("sky", "blue", 12, Instant.ofEpochMilli(0L)), +                 new GameActionInfo("navy", "blue", 3, Instant.ofEpochMilli(0L)), +                 new GameActionInfo("navy", "blue", 3, Instant.ofEpochMilli(0L).plus(Duration.standardMinutes(3))))    // Move the watermark past the end the end of the window - .advanceWatermarkTo(new Instant(0L).plus(TEAM_WINDOW_DURATION) + .advanceWatermarkTo(Instant.ofEpochMilli(0L).plus(TEAM_WINDOW_DURATION)                                 .plus(Duration.standardMinutes(1))) .advanceWatermarkToInfinity(); @@ -161,10 +161,10 @@ the window {{< highlight java >}} TestStream createEvents = TestStream.create(AvroCoder.of(GameActionInfo.class)) - .addElements(new GameActionInfo("sky", "blue", 3, new Instant(0L)), -         new GameActionInfo("navy", "blue", 3, new Instant(0L).plus(Duration.standardMinutes(3)))) + .addElements(new GameActionInfo("sky", "blue", 3, Instant.ofEpochMilli(0L)), +         new GameActionInfo("navy", "blue", 3, Instant.ofEpochMilli(0L).plus(Duration.standardMinutes(3))))    // Move the watermark up to "near" the end of the window - .advanceWatermarkTo(new Instant(0L).plus(TEAM_WINDOW_DURATION) + .advanceWatermarkTo(Instant.ofEpochMilli(0L).plus(TEAM_WINDOW_DURATION)                                 .minus(Duration.standardMinutes(1))) .addElements(new GameActionInfo("sky", "blue", 12, Duration.ZERO)) .advanceWatermarkToInfinity(); @@ -191,10 +191,10 @@ pane, and then after the late data arrives, a pane that refines the result. {{< highlight java >}} TestStream createEvents = TestStream.create(AvroCoder.of(GameActionInfo.class)) - .addElements(new GameActionInfo("sky", "blue", 3, new Instant(0L)), -           new GameActionInfo("navy", "blue", 3, new Instant(0L).plus(Duration.standardMinutes(3)))) + .addElements(new GameActionInfo("sky", "blue", 3, Instant.ofEpochMilli(0L)), +           new GameActionInfo("navy", "blue", 3, Instant.ofEpochMilli(0L).plus(Duration.standardMinutes(3))))     // Move the watermark up to "near" the end of the window - .advanceWatermarkTo(new Instant(0L).plus(TEAM_WINDOW_DURATION) + .advanceWatermarkTo(Instant.ofEpochMilli(0L).plus(TEAM_WINDOW_DURATION)                                  .minus(Duration.standardMinutes(1))) .addElements(new GameActionInfo("sky", "blue", 12, Duration.ZERO)) .advanceWatermarkToInfinity(); @@ -228,14 +228,14 @@ TestStream createEvents = TestStream.create(AvroCoder.of(GameAct .addElements(new GameActionInfo("sky", "blue", 3, Duration.ZERO),          new GameActionInfo("navy", "blue", 3, Duration.standardMinutes(3)))     // Move the watermark up to "near" the end of the window - .advanceWatermarkTo(new Instant(0).plus(TEAM_WINDOW_DURATION) + .advanceWatermarkTo(Instant.ofEpochMilli(0).plus(TEAM_WINDOW_DURATION)                                         .plus(ALLOWED_LATENESS)                                         .plus(Duration.standardMinutes(1))) .addElements(new GameActionInfo(                      "sky",                      "blue",                      12, -                     new Instant(0).plus(TEAM_WINDOW_DURATION).minus(Duration.standardMinutes(1)))) +                     Instant.ofEpochMilli(0).plus(TEAM_WINDOW_DURATION).minus(Duration.standardMinutes(1)))) .advanceWatermarkToInfinity(); PCollection> teamScores = p.apply(createEvents) @@ -261,11 +261,11 @@ apply `CalculateUserScores` {{< highlight java >}} TestStream createEvents = TestStream.create(AvroCoder.of(GameActionInfo.class)) -    .addElements(new GameActionInfo("scarlet", "red", 3, new Instant(0L)), -                new GameActionInfo("scarlet", "red", 2, new Instant(0L).plus(Duration.standardMinutes(1)))) +    .addElements(new GameActionInfo("scarlet", "red", 3, Instant.ofEpochMilli(0L)), +                new GameActionInfo("scarlet", "red", 2, Instant.ofEpochMilli(0L).plus(Duration.standardMinutes(1)))) .advanceProcessingTime(Duration.standardMinutes(12)) -    .addElements(new GameActionInfo("oxblood", "red", 2, new Instant(0L)).plus(Duration.standardSeconds(22)), -                new GameActionInfo("scarlet", "red", 4, new Instant(0L).plus(Duration.standardMinutes(2)))) +    .addElements(new GameActionInfo("oxblood", "red", 2, Instant.ofEpochMilli(0L)).plus(Duration.standardSeconds(22)), +                new GameActionInfo("scarlet", "red", 4, Instant.ofEpochMilli(0L).plus(Duration.standardMinutes(2)))) .advanceProcessingTime(Duration.standardMinutes(15)) .advanceWatermarkToInfinity(); diff --git a/website/www/site/content/en/documentation/patterns/custom-windows.md b/website/www/site/content/en/documentation/patterns/custom-windows.md index 257ce7585487..9c9b8190f319 100644 --- a/website/www/site/content/en/documentation/patterns/custom-windows.md +++ b/website/www/site/content/en/documentation/patterns/custom-windows.md @@ -67,12 +67,12 @@ The following test data tallies two users' scores with and without the `gap` att ``` .apply("Create data", Create.timestamped( - TimestampedValue.of("{\"user\":\"user-1\",\"score\":\"12\",\"gap\":\"5\"}", new Instant()), - TimestampedValue.of("{\"user\":\"user-2\",\"score\":\"4\"}", new Instant()), - TimestampedValue.of("{\"user\":\"user-1\",\"score\":\"-3\",\"gap\":\"5\"}", new Instant().plus(2000)), - TimestampedValue.of("{\"user\":\"user-1\",\"score\":\"2\",\"gap\":\"5\"}", new Instant().plus(9000)), - TimestampedValue.of("{\"user\":\"user-1\",\"score\":\"7\",\"gap\":\"5\"}", new Instant().plus(12000)), - TimestampedValue.of("{\"user\":\"user-2\",\"score\":\"10\"}", new Instant().plus(12000))) + TimestampedValue.of("{\"user\":\"user-1\",\"score\":\"12\",\"gap\":\"5\"}", Instant.now()), + TimestampedValue.of("{\"user\":\"user-2\",\"score\":\"4\"}", Instant.now()), + TimestampedValue.of("{\"user\":\"user-1\",\"score\":\"-3\",\"gap\":\"5\"}", Instant.now().plus(2000)), + TimestampedValue.of("{\"user\":\"user-1\",\"score\":\"2\",\"gap\":\"5\"}", Instant.now().plus(9000)), + TimestampedValue.of("{\"user\":\"user-1\",\"score\":\"7\",\"gap\":\"5\"}", Instant.now().plus(12000)), + TimestampedValue.of("{\"user\":\"user-2\",\"score\":\"10\"}", Instant.now().plus(12000))) .withCoder(StringUtf8Coder.of())) ``` diff --git a/website/www/site/content/en/documentation/programming-guide.md b/website/www/site/content/en/documentation/programming-guide.md index 13900f3a7ceb..dd8ec277ff56 100644 --- a/website/www/site/content/en/documentation/programming-guide.md +++ b/website/www/site/content/en/documentation/programming-guide.md @@ -7062,7 +7062,7 @@ perUser.apply(ParDo.of(new DoFn, OutputT>() { // we would keep resetting the timer to the future). If there is no timer set, then set one to expire in a minute. Long timerTimestampMs = timerTimestamp.read(); Instant timerToSet = (timerTimestamp.isEmpty().read()) - ? Instant.now().plus(Duration.standardMinutes(1)) : new Instant(timerTimestampMs); + ? Instant.now().plus(Duration.standardMinutes(1)) : Instant.ofEpochMilli(timerTimestampMs); // Setting the outputTimestamp to the minimum timestamp in the bag holds the watermark to that timestamp until the // timer fires. This allows outputting all the elements with their timestamp. timer.withOutputTimestamp(minTimestamp.read()).s et(timerToSet). diff --git a/website/www/site/content/en/get-started/wordcount-example.md b/website/www/site/content/en/get-started/wordcount-example.md index e91226c247ca..f1139c7b8057 100644 --- a/website/www/site/content/en/get-started/wordcount-example.md +++ b/website/www/site/content/en/get-started/wordcount-example.md @@ -1265,14 +1265,14 @@ static class AddTimestampFn extends DoFn { @ProcessElement public void processElement(ProcessContext c) { Instant randomTimestamp = - new Instant( + Instant.ofEpochMilli( ThreadLocalRandom.current() .nextLong(minTimestamp.getMillis(), maxTimestamp.getMillis())); /** * Concept #2: Set the data element with that timestamp. */ - c.outputWithTimestamp(c.element(), new Instant(randomTimestamp)); + c.outputWithTimestamp(c.element(), Instant.ofEpochMilli(randomTimestamp)); } } {{< /highlight >}}