def test_rewindow_regroup(self): with TestPipeline() as p: grouped = (p | Create(range(5)) | Map(lambda t: TimestampedValue(('key', t), t)) | 'window' >> WindowInto(FixedWindows(5, offset=3)) | GroupByKey() | MapTuple(lambda k, vs: (k, sorted(vs)))) # Both of these group-and-ungroup sequences should be idempotent. regrouped1 = (grouped | 'w1' >> WindowInto(FixedWindows(5, offset=3)) | 'g1' >> GroupByKey() | FlatMapTuple(lambda k, vs: [(k, v) for v in vs])) regrouped2 = (grouped | FlatMapTuple(lambda k, vs: [(k, v) for v in vs]) | 'w2' >> WindowInto(FixedWindows(5, offset=3)) | 'g2' >> GroupByKey() | MapTuple(lambda k, vs: (k, sorted(vs)))) with_windows = Map(lambda e, w=beam.DoFn.WindowParam: (e, w)) expected = [(('key', [0, 1, 2]), IntervalWindow(-2, 3)), (('key', [3, 4]), IntervalWindow(3, 8))] assert_that(grouped | 'ww' >> with_windows, equal_to(expected)) assert_that( regrouped1 | 'ww1' >> with_windows, equal_to(expected), label='r1') assert_that( regrouped2 | 'ww2' >> with_windows, equal_to(expected), label='r2')
def test_window_assignment_idempotency(self): with TestPipeline() as p: pcoll = self.timestamped_key_values(p, 'key', 0, 2, 4) result = (pcoll | 'window' >> WindowInto(FixedWindows(2)) | 'same window' >> WindowInto(FixedWindows(2)) | 'same window again' >> WindowInto(FixedWindows(2)) | GroupByKey()) assert_that(result, equal_to([('key', [0]), ('key', [2]), ('key', [4])]))
def test_window_assignment_through_multiple_gbk_idempotency(self): with TestPipeline() as p: pcoll = self.timestamped_key_values(p, 'key', 0, 2, 4) result = (pcoll | 'window' >> WindowInto(FixedWindows(2)) | 'gbk' >> GroupByKey() | 'same window' >> WindowInto(FixedWindows(2)) | 'another gbk' >> GroupByKey() | 'same window again' >> WindowInto(FixedWindows(2)) | 'gbk again' >> GroupByKey()) assert_that(result, equal_to([('key', [[[0]]]), ('key', [[[2]]]), ('key', [[[4]]])]))
def test_rewindow(self): with TestPipeline() as p: result = (p | Create([(k, k) for k in range(10)]) | Map(lambda x_t1: TimestampedValue(x_t1[0], x_t1[1])) | 'window' >> WindowInto(SlidingWindows(period=2, size=6)) # Per the model, each element is now duplicated across # three windows. Rewindowing must preserve this duplication. | 'rewindow' >> WindowInto(FixedWindows(5)) | 'rewindow2' >> WindowInto(FixedWindows(5)) | Map(lambda v: ('key', v)) | GroupByKey()) assert_that(result, equal_to([('key', sorted([0, 1, 2, 3, 4] * 3)), ('key', sorted([5, 6, 7, 8, 9] * 3))]))
def test_to_list_and_to_dict(self): with TestPipeline() as pipeline: the_list = [6, 3, 1, 1, 9, 1, 5, 2, 0, 6] timestamp = 0 pcoll = pipeline | 'start' >> Create(the_list) result = pcoll | 'to list' >> combine.ToList() # Now for global combines without default timestamped = pcoll | Map(lambda x: TimestampedValue(x, timestamp)) windowed = timestamped | 'window' >> WindowInto(FixedWindows(60)) result_windowed = ( windowed | 'to list wo defaults' >> combine.ToList().without_defaults()) def matcher(expected): def match(actual): equal_to(expected[0])(actual[0]) return match assert_that(result, matcher([the_list])) assert_that(result_windowed, matcher([the_list]), label='to-list-wo-defaults') with TestPipeline() as pipeline: pairs = [(1, 2), (3, 4), (5, 6)] timestamp = 0 pcoll = pipeline | 'start-pairs' >> Create(pairs) result = pcoll | 'to dict' >> combine.ToDict() # Now for global combines without default timestamped = pcoll | Map(lambda x: TimestampedValue(x, timestamp)) windowed = timestamped | 'window' >> WindowInto(FixedWindows(60)) result_windowed = ( windowed | 'to dict wo defaults' >> combine.ToDict().without_defaults()) def matcher(): def match(actual): equal_to([1])([len(actual)]) equal_to(pairs)(actual[0].items()) return match assert_that(result, matcher()) assert_that(result_windowed, matcher(), label='to-dict-wo-defaults')
def test_global_sample(self): def is_good_sample(actual): assert len(actual) == 1 assert sorted(actual[0]) in [[1, 1, 2], [1, 2, 2]], actual with TestPipeline() as pipeline: timestamp = 0 pcoll = pipeline | 'start' >> Create([1, 1, 2, 2]) # Now for global combines without default timestamped = pcoll | Map(lambda x: TimestampedValue(x, timestamp)) windowed = timestamped | 'window' >> WindowInto(FixedWindows(60)) for ix in range(9): assert_that( pcoll | 'sample-%d' % ix >> combine.Sample.FixedSizeGlobally(3), is_good_sample, label='check-%d' % ix) result_windowed = ( windowed | 'sample-wo-defaults-%d' % ix >> combine.Sample.FixedSizeGlobally(3).without_defaults()) assert_that(result_windowed, is_good_sample, label='check-wo-defaults-%d' % ix)
def test_timestamped_with_combiners(self): with TestPipeline() as p: result = (p # Create some initial test values. | 'start' >> Create([(k, k) for k in range(10)]) # The purpose of the WindowInto transform is to establish a # FixedWindows windowing function for the PCollection. # It does not bucket elements into windows since the timestamps # from Create are not spaced 5 ms apart and very likely they all # fall into the same window. | 'w' >> WindowInto(FixedWindows(5)) # Generate timestamped values using the values as timestamps. # Now there are values 5 ms apart and since Map propagates the # windowing function from input to output the output PCollection # will have elements falling into different 5ms windows. | Map(lambda x_t2: TimestampedValue(x_t2[0], x_t2[1])) # We add a 'key' to each value representing the index of the # window. This is important since there is no guarantee of # order for the elements of a PCollection. | Map(lambda v: (v // 5, v))) # Sum all elements associated with a key and window. Although it # is called CombinePerKey it is really CombinePerKeyAndWindow the # same way GroupByKey is really GroupByKeyAndWindow. sum_per_window = result | CombinePerKey(sum) # Compute mean per key and window. mean_per_window = result | combiners.Mean.PerKey() assert_that(sum_per_window, equal_to([(0, 10), (1, 35)]), label='assert:sum') assert_that(mean_per_window, equal_to([(0, 2.0), (1, 7.0)]), label='assert:mean')
def _test(self, trigger, lateness, expected): windowing = WindowInto( GlobalWindows(), trigger=trigger, accumulation_mode=AccumulationMode.ACCUMULATING, allowed_lateness=lateness).windowing self.assertEqual(trigger.may_lose_data(windowing), expected)
def test_globally(self): l = [ window.TimestampedValue(3, 100), window.TimestampedValue(1, 200), window.TimestampedValue(2, 300) ] with TestPipeline() as p: # Map(lambda x: x) PTransform is added after Create here, because when # a PCollection of TimestampedValues is created with Create PTransform, # the timestamps are not assigned to it. Adding a Map forces the # PCollection to go through a DoFn so that the PCollection consists of # the elements with timestamps assigned to them instead of a PCollection # of TimestampedValue(element, timestamp). pcoll = p | Create(l) | Map(lambda x: x) latest = pcoll | combine.Latest.Globally() assert_that(latest, equal_to([2])) # Now for global combines without default windowed = pcoll | 'window' >> WindowInto(FixedWindows(180)) result_windowed = ( windowed | 'latest wo defaults' >> combine.Latest.Globally().without_defaults()) assert_that(result_windowed, equal_to([3, 2]), label='latest-wo-defaults')
def test_timestamped_value(self): with TestPipeline() as p: result = (p | 'start' >> Create([(k, k) for k in range(10)]) | Map(lambda x_t: TimestampedValue(x_t[0], x_t[1])) | 'w' >> WindowInto(FixedWindows(5)) | Map(lambda v: ('key', v)) | GroupByKey()) assert_that(result, equal_to([('key', [0, 1, 2, 3, 4]), ('key', [5, 6, 7, 8, 9])]))
def test_sliding_windows(self): with TestPipeline() as p: pcoll = self.timestamped_key_values(p, 'key', 1, 2, 3) result = (pcoll | 'w' >> WindowInto(SlidingWindows(period=2, size=4)) | GroupByKey() | reify_windows) expected = [('key @ [-2.0, 2.0)', [1]), ('key @ [0.0, 4.0)', [1, 2, 3]), ('key @ [2.0, 6.0)', [2, 3])] assert_that(result, equal_to(expected))
def test_sessions(self): with TestPipeline() as p: pcoll = self.timestamped_key_values(p, 'key', 1, 2, 3, 20, 35, 27) result = (pcoll | 'w' >> WindowInto(Sessions(10)) | GroupByKey() | sort_values | reify_windows) expected = [('key @ [1.0, 13.0)', [1, 2, 3]), ('key @ [20.0, 45.0)', [20, 27, 35])] assert_that(result, equal_to(expected))
def test_custom_windows(self): with TestPipeline() as p: pcoll = self.timestamped_key_values(p, 'key', 0, 1, 2, 3, 4, 5, 6) # pylint: disable=abstract-class-instantiated result = (pcoll | 'custom window' >> WindowInto(TestCustomWindows()) | GroupByKey() | 'sort values' >> MapTuple(lambda k, vs: (k, sorted(vs)))) assert_that(result, equal_to([('key', [0, 1, 2]), ('key', [3, 4]), ('key', [5]), ('key', [6])]))
def test_window_param(self): class TestDoFn(DoFn): def process(self, element, window=DoFn.WindowParam): yield (element, (float(window.start), float(window.end))) pipeline = TestPipeline() pcoll = (pipeline | Create([1, 7]) | Map(lambda x: TimestampedValue(x, x)) | WindowInto(windowfn=SlidingWindows(10, 5)) | ParDo(TestDoFn())) assert_that(pcoll, equal_to([(1, (-5, 5)), (1, (0, 10)), (7, (0, 10)), (7, (5, 15))])) pipeline.run()
def test_top(self): with TestPipeline() as pipeline: timestamp = 0 # First for global combines. pcoll = pipeline | 'start' >> Create( [6, 3, 1, 1, 9, 1, 5, 2, 0, 6]) result_top = pcoll | 'top' >> combine.Top.Largest(5) result_bot = pcoll | 'bot' >> combine.Top.Smallest(4) assert_that(result_top, equal_to([[9, 6, 6, 5, 3]]), label='assert:top') assert_that(result_bot, equal_to([[0, 1, 1, 1]]), label='assert:bot') # Now for global combines without default timestamped = pcoll | Map(lambda x: TimestampedValue(x, timestamp)) windowed = timestamped | 'window' >> WindowInto(FixedWindows(60)) result_windowed_top = windowed | 'top-wo-defaults' >> combine.Top.Largest( 5, has_defaults=False) result_windowed_bot = (windowed | 'bot-wo-defaults' >> combine.Top.Smallest( 4, has_defaults=False)) assert_that(result_windowed_top, equal_to([[9, 6, 6, 5, 3]]), label='assert:top-wo-defaults') assert_that(result_windowed_bot, equal_to([[0, 1, 1, 1]]), label='assert:bot-wo-defaults') # Again for per-key combines. pcoll = pipeline | 'start-perkey' >> Create( [('a', x) for x in [6, 3, 1, 1, 9, 1, 5, 2, 0, 6]]) result_key_top = pcoll | 'top-perkey' >> combine.Top.LargestPerKey( 5) result_key_bot = pcoll | 'bot-perkey' >> combine.Top.SmallestPerKey( 4) assert_that(result_key_top, equal_to([('a', [9, 6, 6, 5, 3])]), label='key:top') assert_that(result_key_bot, equal_to([('a', [0, 1, 1, 1])]), label='key:bot')
def test_builtin_combines(self): with TestPipeline() as pipeline: vals = [6, 3, 1, 1, 9, 1, 5, 2, 0, 6] mean = sum(vals) / float(len(vals)) size = len(vals) timestamp = 0 # First for global combines. pcoll = pipeline | 'start' >> Create(vals) result_mean = pcoll | 'mean' >> combine.Mean.Globally() result_count = pcoll | 'count' >> combine.Count.Globally() assert_that(result_mean, equal_to([mean]), label='assert:mean') assert_that(result_count, equal_to([size]), label='assert:size') # Now for global combines without default timestamped = pcoll | Map(lambda x: TimestampedValue(x, timestamp)) windowed = timestamped | 'window' >> WindowInto(FixedWindows(60)) result_windowed_mean = (windowed | 'mean-wo-defaults' >> combine.Mean.Globally().without_defaults()) assert_that(result_windowed_mean, equal_to([mean]), label='assert:mean-wo-defaults') result_windowed_count = ( windowed | 'count-wo-defaults' >> combine.Count.Globally().without_defaults()) assert_that(result_windowed_count, equal_to([size]), label='assert:count-wo-defaults') # Again for per-key combines. pcoll = pipeline | 'start-perkey' >> Create([('a', x) for x in vals]) result_key_mean = pcoll | 'mean-perkey' >> combine.Mean.PerKey() result_key_count = pcoll | 'count-perkey' >> combine.Count.PerKey() assert_that(result_key_mean, equal_to([('a', mean)]), label='key:mean') assert_that(result_key_count, equal_to([('a', size)]), label='key:size')
def test_to_set(self): pipeline = TestPipeline() the_list = [6, 3, 1, 1, 9, 1, 5, 2, 0, 6] timestamp = 0 pcoll = pipeline | 'start' >> Create(the_list) result = pcoll | 'to set' >> combine.ToSet() # Now for global combines without default timestamped = pcoll | Map(lambda x: TimestampedValue(x, timestamp)) windowed = timestamped | 'window' >> WindowInto(FixedWindows(60)) result_windowed = ( windowed | 'to set wo defaults' >> combine.ToSet().without_defaults()) def matcher(expected): def match(actual): equal_to(expected[0])(actual[0]) return match assert_that(result, matcher(set(the_list))) assert_that( result_windowed, matcher(set(the_list)), label='to-set-wo-defaults')