github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/testing/benchmarks/nexmark/queries/query5.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 """ 19 Query 5, 'Hot Items'. Which auctions have seen the most bids in the last hour 20 (updated every minute). In CQL syntax:: 21 22 SELECT Rstream(auction) 23 FROM (SELECT B1.auction, count(*) AS num 24 FROM Bid [RANGE 60 MINUTE SLIDE 1 MINUTE] B1 25 GROUP BY B1.auction) 26 WHERE num >= ALL (SELECT count(*) 27 FROM Bid [RANGE 60 MINUTE SLIDE 1 MINUTE] B2 28 GROUP BY B2.auction); 29 30 To make things a bit more dynamic and easier to test we use much shorter 31 windows, and we'll also preserve the bid counts. 32 """ 33 34 import apache_beam as beam 35 from apache_beam.testing.benchmarks.nexmark.queries import nexmark_query_util 36 from apache_beam.testing.benchmarks.nexmark.queries.nexmark_query_util import ResultNames 37 from apache_beam.transforms import window 38 39 40 def load(events, metadata=None, pipeline_options=None): 41 return ( 42 events 43 | nexmark_query_util.JustBids() 44 | 'query5_sliding_window' >> beam.WindowInto( 45 window.SlidingWindows( 46 metadata.get('window_size_sec'), 47 metadata.get('window_period_sec'))) 48 # project out only the auction id for each bid 49 | 'extract_bid_auction' >> beam.Map(lambda bid: bid.auction) 50 | 'bid_count_per_auction' >> beam.combiners.Count.PerElement() 51 | 'bid_max_count' >> beam.CombineGlobally( 52 MostBidCombineFn()).without_defaults() 53 # TODO(leiyiz): fanout with sliding window produces duplicated results, 54 # uncomment after it is fixed 55 # [https://github.com/apache/beam/issues/20528] 56 # .with_fanout(metadata.get('fanout')) 57 | beam.FlatMap( 58 lambda auc_count: [{ 59 ResultNames.AUCTION_ID: auction, ResultNames.NUM: auc_count[1] 60 } for auction in auc_count[0]])) 61 62 63 class MostBidCombineFn(beam.CombineFn): 64 """ 65 combiner function to find auctions with most bid counts 66 """ 67 def create_accumulator(self): 68 return [], 0 69 70 def add_input(self, accumulator, element): 71 accu_list, accu_count = accumulator 72 auction, count = element 73 if accu_count < count: 74 return [auction], count 75 elif accu_count > count: 76 return accu_list, accu_count 77 else: 78 accu_list_new = accu_list.copy() 79 accu_list_new.append(auction) 80 return accu_list_new, accu_count 81 82 def merge_accumulators(self, accumulators): 83 max_list = [] 84 max_count = 0 85 for (accu_list, count) in accumulators: 86 if count == max_count: 87 max_list = max_list + accu_list 88 elif count < max_count: 89 continue 90 else: 91 max_list = accu_list 92 max_count = count 93 return max_list, max_count 94 95 def extract_output(self, accumulator): 96 return accumulator