github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/testing/benchmarks/nexmark/queries/query3.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 3, 'Local Item Suggestion'. Who is selling in OR, ID or CA in category 20 10, and for what auction ids? In CQL syntax:: 21 22 SELECT Istream(P.name, P.city, P.state, A.id) 23 FROM Auction A [ROWS UNBOUNDED], Person P [ROWS UNBOUNDED] 24 WHERE A.seller = P.id 25 AND (P.state = `OR' OR P.state = `ID' OR P.state = `CA') 26 AND A.category = 10; 27 28 We'll implement this query to allow 'new auction' events to come before the 29 'new person' events for the auction seller. Those auctions will be stored until 30 the matching person is seen. Then all subsequent auctions for a person will use 31 the stored person record. 32 """ 33 34 import logging 35 import typing 36 37 import apache_beam as beam 38 from apache_beam.testing.benchmarks.nexmark.models import nexmark_model 39 from apache_beam.testing.benchmarks.nexmark.queries import nexmark_query_util 40 from apache_beam.testing.benchmarks.nexmark.queries.nexmark_query_util import ResultNames 41 from apache_beam.transforms import trigger 42 from apache_beam.transforms import userstate 43 from apache_beam.transforms import window 44 from apache_beam.transforms.userstate import on_timer 45 46 47 def load(events, metadata=None, pipeline_options=None): 48 num_events_in_pane = 30 49 windowed_events = ( 50 events 51 | beam.WindowInto( 52 window.GlobalWindows(), 53 trigger=trigger.Repeatedly(trigger.AfterCount(num_events_in_pane)), 54 accumulation_mode=trigger.AccumulationMode.DISCARDING)) 55 auction_by_seller_id = ( 56 windowed_events 57 | nexmark_query_util.JustAuctions() 58 | 'query3_filter_category' >> beam.Filter(lambda auc: auc.category == 10) 59 | 'query3_key_by_seller' >> beam.ParDo( 60 nexmark_query_util.AuctionBySellerFn())) 61 person_by_id = ( 62 windowed_events 63 | nexmark_query_util.JustPerson() 64 | 'query3_filter_region' >> 65 beam.Filter(lambda person: person.state in ['OR', 'ID', 'CA']) 66 | 'query3_key_by_person_id' >> beam.ParDo( 67 nexmark_query_util.PersonByIdFn())) 68 return ({ 69 nexmark_query_util.AUCTION_TAG: auction_by_seller_id, 70 nexmark_query_util.PERSON_TAG: person_by_id, 71 } 72 | beam.CoGroupByKey() 73 | 'query3_join' >> beam.ParDo( 74 JoinFn(metadata.get('max_auction_waiting_time'))) 75 | 'query3_output' >> beam.Map( 76 lambda t: { 77 ResultNames.NAME: t[1].name, 78 ResultNames.CITY: t[1].city, 79 ResultNames.STATE: t[1].state, 80 ResultNames.AUCTION_ID: t[0].id 81 })) 82 83 84 class JoinFn(beam.DoFn): 85 """ 86 Join auctions and person by person id and emit their product one pair at 87 a time. 88 89 We know a person may submit any number of auctions. Thus new person event 90 must have the person record stored in persistent state in order to match 91 future auctions by that person. 92 93 However we know that each auction is associated with at most one person, so 94 only need to store auction records in persistent state until we have seen the 95 corresponding person record. And of course may have already seen that record. 96 """ 97 98 AUCTIONS = 'auctions_state' 99 PERSON = 'person_state' 100 PERSON_EXPIRING = 'person_state_expiring' 101 102 auction_spec = userstate.BagStateSpec(AUCTIONS, nexmark_model.Auction.CODER) 103 person_spec = userstate.ReadModifyWriteStateSpec( 104 PERSON, nexmark_model.Person.CODER) 105 person_timer_spec = userstate.TimerSpec( 106 PERSON_EXPIRING, userstate.TimeDomain.WATERMARK) 107 108 def __init__(self, max_auction_wait_time): 109 self.max_auction_wait_time = max_auction_wait_time 110 111 def process( # type: ignore 112 self, 113 element: typing.Tuple[ 114 str, 115 typing.Dict[str, 116 typing.Union[typing.List[nexmark_model.Auction], 117 typing.List[nexmark_model.Person]]]], 118 auction_state=beam.DoFn.StateParam(auction_spec), 119 person_state=beam.DoFn.StateParam(person_spec), 120 person_timer=beam.DoFn.TimerParam(person_timer_spec)): 121 # extract group with tags from element tuple 122 _, group = element 123 124 existing_person = person_state.read() 125 if existing_person: 126 # the person exists in person_state for this person id 127 for auction in group[nexmark_query_util.AUCTION_TAG]: 128 yield auction, existing_person 129 return 130 131 new_person = None 132 for person in group[nexmark_query_util.PERSON_TAG]: 133 if not new_person: 134 new_person = person 135 else: 136 logging.error( 137 'two new person wtih same key: %s and %s' % (person, new_person)) 138 continue 139 # read all pending auctions for this person id, output and flush it 140 pending_auctions = auction_state.read() 141 if pending_auctions: 142 for pending_auction in pending_auctions: 143 yield pending_auction, new_person 144 auction_state.clear() 145 # output new auction for this person id 146 for auction in group[nexmark_query_util.AUCTION_TAG]: 147 yield auction, new_person 148 # remember person for max_auction_wait_time seconds for future auctions 149 person_state.write(new_person) 150 person_timer.set(new_person.date_time + self.max_auction_wait_time) 151 # we are done if we have seen a new person 152 if new_person: 153 return 154 155 # remember auction until we see person 156 for auction in group[nexmark_query_util.AUCTION_TAG]: 157 auction_state.add(auction) 158 159 @on_timer(person_timer_spec) 160 def expiry(self, person_state=beam.DoFn.StateParam(person_spec)): 161 person_state.clear()