github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/testing/benchmarks/nexmark/queries/query8.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 8, 'Monitor New Users'. Select people who have entered the system and 20 created auctions in the last 12 hours, updated every 12 hours. In CQL syntax:: 21 22 SELECT Rstream(P.id, P.name, A.reserve) 23 FROM Person [RANGE 12 HOUR] P, Auction [RANGE 12 HOUR] A 24 WHERE P.id = A.seller; 25 26 To make things a bit more dynamic and easier to test we'll use a much 27 shorter window. 28 """ 29 30 import apache_beam as beam 31 from apache_beam.testing.benchmarks.nexmark.queries import nexmark_query_util 32 from apache_beam.testing.benchmarks.nexmark.queries.nexmark_query_util import ResultNames 33 from apache_beam.transforms import window 34 35 36 def load(events, metadata=None, pipeline_options=None): 37 # window person and key by persons' id 38 persons_by_id = ( 39 events 40 | nexmark_query_util.JustPerson() 41 | 'query8_window_person' >> beam.WindowInto( 42 window.FixedWindows(metadata.get('window_size_sec'))) 43 | 'query8_person_by_id' >> beam.ParDo(nexmark_query_util.PersonByIdFn())) 44 # window auction and key by auctions' seller 45 auctions_by_seller = ( 46 events 47 | nexmark_query_util.JustAuctions() 48 | 'query8_window_auction' >> beam.WindowInto( 49 window.FixedWindows(metadata.get('window_size_sec'))) 50 | 'query8_auction_by_seller' >> beam.ParDo( 51 nexmark_query_util.AuctionBySellerFn())) 52 return ({ 53 nexmark_query_util.PERSON_TAG: persons_by_id, 54 nexmark_query_util.AUCTION_TAG: auctions_by_seller 55 } 56 | beam.CoGroupByKey() 57 | 'query8_join' >> beam.ParDo(JoinPersonAuctionFn())) 58 59 60 class JoinPersonAuctionFn(beam.DoFn): 61 def process(self, element): 62 _, group = element 63 persons = group[nexmark_query_util.PERSON_TAG] 64 person = persons[0] if persons else None 65 if person is None: 66 # do nothing if this seller id is not a new person in this window 67 return 68 for auction in group[nexmark_query_util.AUCTION_TAG]: 69 yield { 70 ResultNames.ID: person.id, 71 ResultNames.NAME: person.name, 72 ResultNames.RESERVE: auction.reserve 73 }