github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/testing/benchmarks/nexmark/queries/query12.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 12, How many bids does a user make within a fixed processing time limit 20 (Not in original suite.) 21 22 Group bids by the same user into processing time windows of window_size_sec. 23 Emit the count of bids per window. 24 """ 25 26 import apache_beam as beam 27 from apache_beam.testing.benchmarks.nexmark.queries import nexmark_query_util 28 from apache_beam.testing.benchmarks.nexmark.queries.nexmark_query_util import ResultNames 29 from apache_beam.transforms import trigger 30 from apache_beam.transforms import window 31 32 33 def load(events, metadata=None, pipeline_options=None): 34 return ( 35 events 36 | nexmark_query_util.JustBids() 37 | 'query12_extract_bidder' >> beam.Map(lambda bid: bid.bidder) 38 # windowing with processing time trigger, currently not supported in batch 39 | beam.WindowInto( 40 window.GlobalWindows(), 41 trigger=trigger.Repeatedly( 42 trigger.AfterProcessingTime(metadata.get('window_size_sec'))), 43 accumulation_mode=trigger.AccumulationMode.DISCARDING, 44 allowed_lateness=0) 45 | 'query12_bid_count' >> beam.combiners.Count.PerElement() 46 | 'query12_output' >> beam.Map( 47 lambda t: { 48 ResultNames.BIDDER_ID: t[0], ResultNames.BID_COUNT: t[1] 49 }))