| /* |
| * Licensed to the Apache Software Foundation (ASF) under one |
| * or more contributor license agreements. See the NOTICE file |
| * distributed with this work for additional information |
| * regarding copyright ownership. The ASF licenses this file |
| * to you under the Apache License, Version 2.0 (the |
| * "License"); you may not use this file except in compliance |
| * with the License. You may obtain a copy of the License at |
| * |
| * http://www.apache.org/licenses/LICENSE-2.0 |
| * |
| * Unless required by applicable law or agreed to in writing, software |
| * distributed under the License is distributed on an "AS IS" BASIS, |
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| * See the License for the specific language governing permissions and |
| * limitations under the License. |
| */ |
| package org.apache.beam.runners.spark.structuredstreaming.translation.batch; |
| |
| import java.io.Serializable; |
| import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingPipelineOptions; |
| import org.apache.beam.runners.spark.structuredstreaming.SparkStructuredStreamingRunner; |
| import org.apache.beam.sdk.Pipeline; |
| import org.apache.beam.sdk.options.PipelineOptionsFactory; |
| import org.apache.beam.sdk.testing.PAssert; |
| import org.apache.beam.sdk.transforms.Create; |
| import org.apache.beam.sdk.transforms.Sum; |
| import org.apache.beam.sdk.transforms.windowing.FixedWindows; |
| import org.apache.beam.sdk.transforms.windowing.Window; |
| import org.apache.beam.sdk.values.PCollection; |
| import org.apache.beam.sdk.values.TimestampedValue; |
| import org.joda.time.Duration; |
| import org.joda.time.Instant; |
| import org.junit.BeforeClass; |
| import org.junit.Test; |
| import org.junit.runner.RunWith; |
| import org.junit.runners.JUnit4; |
| |
| /** Test class for beam to spark window assign translation. */ |
| @RunWith(JUnit4.class) |
| public class WindowAssignTest implements Serializable { |
| private static Pipeline pipeline; |
| |
| @BeforeClass |
| public static void beforeClass() { |
| SparkStructuredStreamingPipelineOptions options = |
| PipelineOptionsFactory.create().as(SparkStructuredStreamingPipelineOptions.class); |
| options.setRunner(SparkStructuredStreamingRunner.class); |
| options.setTestMode(true); |
| pipeline = Pipeline.create(options); |
| } |
| |
| @Test |
| public void testWindowAssign() { |
| PCollection<Integer> input = |
| 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)))) |
| .apply(Window.into(FixedWindows.of(Duration.millis(10)))) |
| .apply(Sum.integersGlobally().withoutDefaults()); |
| PAssert.that(input).containsInAnyOrder(6, 9); |
| pipeline.run(); |
| } |
| } |