github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/testing/util_test.py (about) 1 # 2 # Licensed to the Apache Software Foundation (ASF) under one or more 3 # contributor license agreements. See the NOTICE file distributed with 4 # this work for additional information regarding copyright ownership. 5 # The ASF licenses this file to You under the Apache License, Version 2.0 6 # (the "License"); you may not use this file except in compliance with 7 # the License. You may obtain a copy of the License at 8 # 9 # http://www.apache.org/licenses/LICENSE-2.0 10 # 11 # Unless required by applicable law or agreed to in writing, software 12 # distributed under the License is distributed on an "AS IS" BASIS, 13 # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 # See the License for the specific language governing permissions and 15 # limitations under the License. 16 # 17 18 """Unit tests for testing utilities.""" 19 20 # pytype: skip-file 21 22 import unittest 23 24 import apache_beam as beam 25 from apache_beam import Create 26 from apache_beam.options.pipeline_options import StandardOptions 27 from apache_beam.testing.test_pipeline import TestPipeline 28 from apache_beam.testing.util import BeamAssertException 29 from apache_beam.testing.util import TestWindowedValue 30 from apache_beam.testing.util import assert_that 31 from apache_beam.testing.util import equal_to 32 from apache_beam.testing.util import equal_to_per_window 33 from apache_beam.testing.util import is_empty 34 from apache_beam.testing.util import is_not_empty 35 from apache_beam.transforms import trigger 36 from apache_beam.transforms import window 37 from apache_beam.transforms.window import FixedWindows 38 from apache_beam.transforms.window import GlobalWindow 39 from apache_beam.transforms.window import IntervalWindow 40 from apache_beam.utils.timestamp import MIN_TIMESTAMP 41 42 43 class UtilTest(unittest.TestCase): 44 def test_assert_that_passes(self): 45 with TestPipeline() as p: 46 assert_that(p | Create([1, 2, 3]), equal_to([1, 2, 3])) 47 48 def test_assert_that_passes_order_does_not_matter(self): 49 with TestPipeline() as p: 50 assert_that(p | Create([1, 2, 3]), equal_to([2, 1, 3])) 51 52 def test_assert_that_passes_order_does_not_matter_with_negatives(self): 53 with TestPipeline() as p: 54 assert_that(p | Create([1, -2, 3]), equal_to([-2, 1, 3])) 55 56 def test_assert_that_passes_empty_equal_to(self): 57 with TestPipeline() as p: 58 assert_that(p | Create([]), equal_to([])) 59 60 def test_assert_that_passes_empty_is_empty(self): 61 with TestPipeline() as p: 62 assert_that(p | Create([]), is_empty()) 63 64 def test_assert_that_fails(self): 65 with self.assertRaises(Exception): 66 with TestPipeline() as p: 67 assert_that(p | Create([1, 10, 100]), equal_to([1, 2, 3])) 68 69 def test_assert_missing(self): 70 with self.assertRaisesRegex(BeamAssertException, 71 r"missing elements \['c'\]"): 72 with TestPipeline() as p: 73 assert_that(p | Create(['a', 'b']), equal_to(['a', 'b', 'c'])) 74 75 def test_assert_unexpected(self): 76 with self.assertRaisesRegex(BeamAssertException, 77 r"unexpected elements \['c', 'd'\]|" 78 r"unexpected elements \['d', 'c'\]"): 79 with TestPipeline() as p: 80 assert_that(p | Create(['a', 'b', 'c', 'd']), equal_to(['a', 'b'])) 81 82 def test_assert_missing_and_unexpected(self): 83 with self.assertRaisesRegex( 84 BeamAssertException, 85 r"unexpected elements \['c'\].*missing elements \['d'\]"): 86 with TestPipeline() as p: 87 assert_that(p | Create(['a', 'b', 'c']), equal_to(['a', 'b', 'd'])) 88 89 def test_assert_with_custom_comparator(self): 90 with TestPipeline() as p: 91 assert_that( 92 p | Create([1, 2, 3]), 93 equal_to(['1', '2', '3'], equals_fn=lambda e, a: int(e) == int(a))) 94 95 def test_reified_value_passes(self): 96 expected = [ 97 TestWindowedValue(v, MIN_TIMESTAMP, [GlobalWindow()]) 98 for v in [1, 2, 3] 99 ] 100 with TestPipeline() as p: 101 assert_that(p | Create([2, 3, 1]), equal_to(expected), reify_windows=True) 102 103 def test_reified_value_assert_fail_unmatched_value(self): 104 expected = [ 105 TestWindowedValue(v + 1, MIN_TIMESTAMP, [GlobalWindow()]) 106 for v in [1, 2, 3] 107 ] 108 with self.assertRaises(Exception): 109 with TestPipeline() as p: 110 assert_that( 111 p | Create([2, 3, 1]), equal_to(expected), reify_windows=True) 112 113 def test_reified_value_assert_fail_unmatched_timestamp(self): 114 expected = [TestWindowedValue(v, 1, [GlobalWindow()]) for v in [1, 2, 3]] 115 with self.assertRaises(Exception): 116 with TestPipeline() as p: 117 assert_that( 118 p | Create([2, 3, 1]), equal_to(expected), reify_windows=True) 119 120 def test_reified_value_assert_fail_unmatched_window(self): 121 expected = [ 122 TestWindowedValue(v, MIN_TIMESTAMP, [IntervalWindow(0, 1)]) 123 for v in [1, 2, 3] 124 ] 125 with self.assertRaises(Exception): 126 with TestPipeline() as p: 127 assert_that( 128 p | Create([2, 3, 1]), equal_to(expected), reify_windows=True) 129 130 def test_assert_that_fails_on_empty_input(self): 131 with self.assertRaises(Exception): 132 with TestPipeline() as p: 133 assert_that(p | Create([]), equal_to([1, 2, 3])) 134 135 def test_assert_that_fails_on_empty_expected(self): 136 with self.assertRaises(Exception): 137 with TestPipeline() as p: 138 assert_that(p | Create([1, 2, 3]), is_empty()) 139 140 def test_assert_that_passes_is_not_empty(self): 141 with TestPipeline() as p: 142 assert_that(p | Create([1, 2, 3]), is_not_empty()) 143 144 def test_assert_that_fails_on_is_not_empty_expected(self): 145 with self.assertRaises(BeamAssertException): 146 with TestPipeline() as p: 147 assert_that(p | Create([]), is_not_empty()) 148 149 def test_equal_to_per_window_passes(self): 150 start = int(MIN_TIMESTAMP.micros // 1e6) - 5 151 end = start + 20 152 expected = { 153 window.IntervalWindow(start, end): [('k', [1])], 154 } 155 with TestPipeline(options=StandardOptions(streaming=True)) as p: 156 assert_that(( 157 p 158 | Create([1]) 159 | beam.WindowInto( 160 FixedWindows(20), 161 trigger=trigger.AfterWatermark(), 162 accumulation_mode=trigger.AccumulationMode.DISCARDING) 163 | beam.Map(lambda x: ('k', x)) 164 | beam.GroupByKey()), 165 equal_to_per_window(expected), 166 reify_windows=True) 167 168 def test_equal_to_per_window_fail_unmatched_window(self): 169 with self.assertRaises(BeamAssertException): 170 expected = { 171 window.IntervalWindow(50, 100): [('k', [1])], 172 } 173 with TestPipeline(options=StandardOptions(streaming=True)) as p: 174 assert_that(( 175 p 176 | Create([1]) 177 | beam.WindowInto( 178 FixedWindows(20), 179 trigger=trigger.AfterWatermark(), 180 accumulation_mode=trigger.AccumulationMode.DISCARDING) 181 | beam.Map(lambda x: ('k', x)) 182 | beam.GroupByKey()), 183 equal_to_per_window(expected), 184 reify_windows=True) 185 186 def test_equal_to_per_window_fail_unmatched_element(self): 187 with self.assertRaises(BeamAssertException): 188 start = int(MIN_TIMESTAMP.micros // 1e6) - 5 189 end = start + 20 190 expected = { 191 window.IntervalWindow(start, end): [('k', [1]), ('k', [2])], 192 } 193 with TestPipeline(options=StandardOptions(streaming=True)) as p: 194 assert_that(( 195 p 196 | Create([1]) 197 | beam.WindowInto( 198 FixedWindows(20), 199 trigger=trigger.AfterWatermark(), 200 accumulation_mode=trigger.AccumulationMode.DISCARDING) 201 | beam.Map(lambda x: ('k', x)) 202 | beam.GroupByKey()), 203 equal_to_per_window(expected), 204 reify_windows=True) 205 206 def test_equal_to_per_window_succeeds_no_reify_windows(self): 207 start = int(MIN_TIMESTAMP.micros // 1e6) - 5 208 end = start + 20 209 expected = { 210 window.IntervalWindow(start, end): [('k', [1])], 211 } 212 with TestPipeline(options=StandardOptions(streaming=True)) as p: 213 assert_that(( 214 p 215 | Create([1]) 216 | beam.WindowInto( 217 FixedWindows(20), 218 trigger=trigger.AfterWatermark(), 219 accumulation_mode=trigger.AccumulationMode.DISCARDING) 220 | beam.Map(lambda x: ('k', x)) 221 | beam.GroupByKey()), 222 equal_to_per_window(expected)) 223 224 def test_equal_to_per_window_fail_unexpected_element(self): 225 with self.assertRaises(BeamAssertException): 226 start = int(MIN_TIMESTAMP.micros // 1e6) - 5 227 end = start + 20 228 expected = { 229 window.IntervalWindow(start, end): [('k', [1])], 230 } 231 with TestPipeline(options=StandardOptions(streaming=True)) as p: 232 assert_that(( 233 p 234 | Create([1, 2]) 235 | beam.WindowInto( 236 FixedWindows(20), 237 trigger=trigger.AfterWatermark(), 238 accumulation_mode=trigger.AccumulationMode.DISCARDING) 239 | beam.Map(lambda x: ('k', x)) 240 | beam.GroupByKey()), 241 equal_to_per_window(expected), 242 reify_windows=True) 243 244 245 if __name__ == '__main__': 246 unittest.main()