github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/testing/benchmarks/nexmark/queries/query7.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 7, 'Highest Bid'. Select the bids with the highest bid price in the 20 last minute. In CQL syntax:: 21 22 SELECT Rstream(B.auction, B.price, B.bidder) 23 FROM Bid [RANGE 1 MINUTE SLIDE 1 MINUTE] B 24 WHERE B.price = (SELECT MAX(B1.price) 25 FROM BID [RANGE 1 MINUTE SLIDE 1 MINUTE] B1); 26 27 We will use a shorter window to help make testing easier. We'll also 28 implement this using a side-input in order to exercise that functionality. 29 (A combiner, as used in Query 5, is a more efficient approach.). 30 """ 31 32 import apache_beam as beam 33 from apache_beam.testing.benchmarks.nexmark.queries import nexmark_query_util 34 from apache_beam.transforms import window 35 36 37 def load(events, metadata=None, pipeline_options=None): 38 # window bids into fixed window 39 sliding_bids = ( 40 events 41 | nexmark_query_util.JustBids() 42 | beam.WindowInto(window.FixedWindows(metadata.get('window_size_sec')))) 43 # find the largest price in all bids per window 44 max_prices = ( 45 sliding_bids 46 | beam.Map(lambda bid: bid.price) 47 | beam.CombineGlobally(max).without_defaults()) 48 return ( 49 sliding_bids 50 | 'select_bids' >> beam.ParDo( 51 SelectMaxBidFn(), beam.pvalue.AsSingleton(max_prices))) 52 53 54 class SelectMaxBidFn(beam.DoFn): 55 def process(self, element, max_bid_price): 56 if element.price == max_bid_price: 57 yield element