github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/testing/benchmarks/nexmark/queries/query11.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 11, How many bids did a user make in each session he was active?
    20  (Not in original suite.)
    21  
    22  Group bids by the same user into sessions with window_size_sec max gap.
    23  However limit the session to at most max_log_events. Emit the number of
    24  bids per session.
    25  """
    26  
    27  import apache_beam as beam
    28  from apache_beam.testing.benchmarks.nexmark.queries import nexmark_query_util
    29  from apache_beam.testing.benchmarks.nexmark.queries.nexmark_query_util import ResultNames
    30  from apache_beam.transforms import trigger
    31  from apache_beam.transforms import window
    32  
    33  
    34  def load(events, metadata=None, pipeline_options=None):
    35  
    36    return (
    37        events
    38        # filter to get only bids and then extract bidder id
    39        | nexmark_query_util.JustBids()
    40        | 'query11_extract_bidder' >> beam.Map(lambda bid: bid.bidder)
    41        # window auction and key by auctions' seller
    42        | 'query11_session_window' >> beam.WindowInto(
    43            window.Sessions(metadata.get('window_size_sec')),
    44            trigger=trigger.AfterWatermark(
    45                early=trigger.AfterCount(metadata.get('max_log_events'))),
    46            accumulation_mode=trigger.AccumulationMode.DISCARDING,
    47            allowed_lateness=metadata.get('occasional_delay_sec') // 2)
    48        # count per bidder
    49        | beam.combiners.Count.PerElement()
    50        | beam.Map(
    51            lambda bidder_count: {
    52                ResultNames.BIDDER_ID: bidder_count[0],
    53                ResultNames.BID_COUNT: bidder_count[1]
    54            }))