pabloem commented on a change in pull request #12580:
URL: https://github.com/apache/beam/pull/12580#discussion_r474419909



##########
File path: sdks/python/apache_beam/testing/benchmarks/nexmark/queries/query7.py
##########
@@ -0,0 +1,59 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+"""
+Query 7, 'Highest Bid'. Select the bids with the highest bid price in the
+last minute. In CQL syntax::
+
+  SELECT Rstream(B.auction, B.price, B.bidder)
+  FROM Bid [RANGE 1 MINUTE SLIDE 1 MINUTE] B
+  WHERE B.price = (SELECT MAX(B1.price)
+                   FROM BID [RANGE 1 MINUTE SLIDE 1 MINUTE] B1);
+
+We will use a shorter window to help make testing easier. We'll also
+implement this using a side-input in order to exercise that functionality.
+(A combiner, as used in Query 5, is a more efficient approach.).
+"""
+
+from __future__ import absolute_import
+
+import apache_beam as beam
+from apache_beam.testing.benchmarks.nexmark.queries import nexmark_query_util
+from apache_beam.transforms import window
+
+
+def load(events, metadata=None):
+  # window bids into fixed window
+  sliding_bids = (
+      events
+      | nexmark_query_util.JustBids()
+      | beam.WindowInto(window.FixedWindows(metadata.get('window_size_sec'))))
+  # find the largest price in all bids per window
+  max_prices = (
+      sliding_bids
+      | beam.Map(lambda bid: bid.price)
+      | beam.CombineGlobally(max).without_defaults())
+  return (
+      sliding_bids
+      | 'select_bids' >> beam.ParDo(
+          SelectMaxBidFn(), beam.pvalue.AsSingleton(max_prices)))

Review comment:
       Is this how this is implemented in Java? I am wondering if we should 
make bids comparable. If they were comparable, then you would be able to just 
return max_prices
   
   e.g.:
   
   ```
   @functools.total_ordering
   class ComparableBidByPrice(object):
     def __init__(self, bid):
       self.bid = bid
     def __eq__(self, other):
       return self.bid == other.bid
     def __lt__(self, other):
       return self.bid.price < other.bid.price
   ```
   
   And then you'd do:
   
   ```
     max_bids = (
         sliding_bids
         | beam.Map(ComparableBidByPrice)
         | beam.CombineGlobally(max).without_defaults())
   ```
   
   thoughts? The main thing here is having one fewer stage, thus higher 
performance - but I think the best option is to do whatever Java does.

##########
File path: 
sdks/python/apache_beam/testing/benchmarks/nexmark/nexmark_launcher.py
##########
@@ -242,12 +249,28 @@ def run(self):
     queries = {
         0: query0,
         1: query1,
-        2: query2,  # TODO(mariagh): Add more queries.
-        9: query9
+        2: query2,
+        3: query3,
+        4: query4,
+        5: query5,
+        6: query6,
+        7: query7,
+        8: query8,
+        9: query9,
+        11: query11
     }
 
     # TODO(mariagh): Move to a config file.

Review comment:
       maybe remove this TODO

##########
File path: sdks/python/apache_beam/testing/benchmarks/nexmark/queries/query6.py
##########
@@ -0,0 +1,92 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+"""
+Query 6, 'Average Selling Price by Seller'. Select the average selling price
+over the last 10 closed auctions by the same seller. In CQL syntax::
+
+  SELECT Istream(AVG(Q.final), Q.seller)
+  FROM (SELECT Rstream(MAX(B.price) AS final, A.seller)
+    FROM Auction A [ROWS UNBOUNDED], Bid B [ROWS UNBOUNDED]
+    WHERE A.id=B.auction
+      AND B.datetime < A.expires AND A.expires < CURRENT_TIME
+    GROUP BY A.id, A.seller) [PARTITION BY A.seller ROWS 10] Q
+  GROUP BY Q.seller;
+"""
+
+from __future__ import absolute_import
+from __future__ import division
+
+import apache_beam as beam
+from apache_beam.testing.benchmarks.nexmark.queries import nexmark_query_util
+from apache_beam.testing.benchmarks.nexmark.queries import winning_bids
+from apache_beam.testing.benchmarks.nexmark.queries.nexmark_query_util import 
ResultNames
+from apache_beam.transforms import trigger
+from apache_beam.transforms import window
+
+
+def load(events, metadata=None):
+  # find winning bids for each closed auction
+  return (
+      events
+      # find winning bids
+      | beam.Filter(nexmark_query_util.auction_or_bid)
+      | winning_bids.WinningBids()
+      # (auction_bids -> (aution.seller, bid)
+      | beam.Map(lambda auc_bid: (auc_bid.auction.seller, auc_bid.bid))
+      # calculate and output mean as data arrives
+      | beam.WindowInto(
+          window.GlobalWindows(),
+          trigger=trigger.Repeatedly(trigger.AfterCount(1)),
+          accumulation_mode=trigger.AccumulationMode.ACCUMULATING,
+          allowed_lateness=0)

Review comment:
       This trigger is a little hard to wrap my head around : ) can you help me 
understand it? So I guess we simply accumulate fired panes and fire everything 
every time? (let's say that the stream contains a new element every second. 
Would we fire 1000 elements after 1000 seconds?)

##########
File path: sdks/python/apache_beam/testing/benchmarks/nexmark/queries/query4.py
##########
@@ -0,0 +1,81 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+"""
+Query 4, 'Average Price for a Category'. Select the average of the wining bid
+prices for all closed auctions in each category. In CQL syntax::
+
+  SELECT Istream(AVG(Q.final))
+  FROM Category C, (SELECT Rstream(MAX(B.price) AS final, A.category)
+    FROM Auction A [ROWS UNBOUNDED], Bid B [ROWS UNBOUNDED]
+    WHERE A.id=B.auction
+      AND B.datetime < A.expires AND A.expires < CURRENT_TIME
+    GROUP BY A.id, A.category) Q
+  WHERE Q.category = C.id
+  GROUP BY C.id;
+
+For extra spiciness our implementation differs slightly from the above:
+
+* We select both the average winning price and the category.
+* We don't bother joining with a static category table, since it's
+  contents are never used.
+* We only consider bids which are above the auction's reserve price.
+* We accept the highest-price, earliest valid bid as the winner.
+* We calculate the averages oven a sliding window of size
+  window_size_sec and period window_period_sec.
+"""
+
+from __future__ import absolute_import
+
+import apache_beam as beam
+from apache_beam.testing.benchmarks.nexmark.queries import nexmark_query_util
+from apache_beam.testing.benchmarks.nexmark.queries import winning_bids
+from apache_beam.testing.benchmarks.nexmark.queries.nexmark_query_util import 
ResultNames
+from apache_beam.transforms import window
+
+
+def load(events, metadata=None):
+  # find winning bids for each closed auction
+  all_winning_bids = (
+      events
+      | beam.Filter(nexmark_query_util.auction_or_bid)
+      | winning_bids.WinningBids())
+  return (
+      all_winning_bids
+      # key winning bids by auction category
+      | beam.Map(lambda auc_bid: (auc_bid.auction.category, auc_bid.bid.price))
+      # re-window for sliding average
+      | beam.WindowInto(
+          window.SlidingWindows(
+              metadata.get('window_size_sec'),
+              metadata.get('window_period_sec')))
+      # average for each category
+      | beam.CombinePerKey(beam.combiners.MeanCombineFn())
+      # TODO(leiyiz): fanout with sliding window produces duplicated results,
+      #   uncomment after it is fixed [BEAM-10617]
+      # .with_hot_key_fanout(metadata.get('fanout'))
+      # produce output
+      | beam.ParDo(ProjectToCategoryPriceFn()))
+
+
+class ProjectToCategoryPriceFn(beam.DoFn):
+  def process(self, element, pane_info=beam.DoFn.PaneInfoParam):
+    yield {
+        ResultNames.CATEGORY: element[0],
+        ResultNames.PRICE: element[1],
+        ResultNames.IS_LAST: pane_info.is_last

Review comment:
       why do we also return whether this is the last pane in a window?

##########
File path: sdks/python/apache_beam/testing/benchmarks/nexmark/queries/query3.py
##########
@@ -0,0 +1,158 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+"""
+Query 3, 'Local Item Suggestion'. Who is selling in OR, ID or CA in category
+10, and for what auction ids? In CQL syntax::
+
+  SELECT Istream(P.name, P.city, P.state, A.id)
+  FROM Auction A [ROWS UNBOUNDED], Person P [ROWS UNBOUNDED]
+  WHERE A.seller = P.id
+    AND (P.state = `OR' OR P.state = `ID' OR P.state = `CA')
+    AND A.category = 10;
+
+We'll implement this query to allow 'new auction' events to come before the
+'new person' events for the auction seller. Those auctions will be stored until
+the matching person is seen. Then all subsequent auctions for a person will use
+the stored person record.
+"""
+
+from __future__ import absolute_import
+
+import logging
+
+import apache_beam as beam
+from apache_beam.testing.benchmarks.nexmark.models import nexmark_model
+from apache_beam.testing.benchmarks.nexmark.queries import nexmark_query_util
+from apache_beam.testing.benchmarks.nexmark.queries.nexmark_query_util import 
ResultNames
+from apache_beam.transforms import trigger
+from apache_beam.transforms import userstate
+from apache_beam.transforms import window
+from apache_beam.transforms.userstate import on_timer
+
+
+def load(events, metadata=None):
+  num_events_in_pane = 30
+  windowed_events = (
+      events
+      | beam.WindowInto(
+          window.GlobalWindows(),
+          trigger=trigger.Repeatedly(trigger.AfterCount(num_events_in_pane)),
+          accumulation_mode=trigger.AccumulationMode.DISCARDING))
+  auction_by_seller_id = (
+      windowed_events
+      | nexmark_query_util.JustAuctions()
+      | 'query3_filter_category' >> beam.Filter(lambda auc: auc.category == 10)
+      | 'query3_key_by_seller' >> beam.ParDo(
+          nexmark_query_util.AuctionBySellerFn()))
+  person_by_id = (
+      windowed_events
+      | nexmark_query_util.JustPerson()
+      | 'query3_filter_region' >>
+      beam.Filter(lambda person: person.state in ['OR', 'ID', 'CA'])
+      | 'query3_key_by_person_id' >> beam.ParDo(
+          nexmark_query_util.PersonByIdFn()))
+  return ({
+      nexmark_query_util.AUCTION_TAG: auction_by_seller_id,
+      nexmark_query_util.PERSON_TAG: person_by_id,
+  }
+          | beam.CoGroupByKey()
+          | 'query3_join' >> beam.ParDo(
+              JoinFn(metadata.get('max_auction_waiting_time')))
+          | 'query3_output' >> beam.Map(
+              lambda t: {
+                  ResultNames.NAME: t[1].name,
+                  ResultNames.CITY: t[1].city,
+                  ResultNames.STATE: t[1].state,
+                  ResultNames.AUCTION_ID: t[0].id
+              }))
+
+
+class JoinFn(beam.DoFn):
+  """
+  Join auctions and person by person id and emit their product one pair at
+  a time.
+
+  We know a person may submit any number of auctions. Thus new person event
+  must have the person record stored in persistent state in order to match
+  future auctions by that person.
+
+  However we know that each auction is associated with at most one person, so
+  only need to store auction records in persistent state until we have seen the
+  corresponding person record. And of course may have already seen that record.
+  """
+
+  AUCTIONS = 'auctions_state'
+  PERSON = 'person_state'
+  PERSON_EXPIRING = 'person_state_expiring'
+
+  auction_spec = userstate.BagStateSpec(AUCTIONS, nexmark_model.Auction.CODER)
+  person_spec = userstate.ReadModifyWriteStateSpec(
+      PERSON, nexmark_model.Person.CODER)
+  person_timer_spec = userstate.TimerSpec(
+      PERSON_EXPIRING, userstate.TimeDomain.WATERMARK)
+
+  def __init__(self, max_auction_wait_time):
+    self.max_auction_wait_time = max_auction_wait_time
+
+  def process(
+      self,
+      element,
+      auction_state=beam.DoFn.StateParam(auction_spec),
+      person_state=beam.DoFn.StateParam(person_spec),
+      person_timer=beam.DoFn.TimerParam(person_timer_spec)):
+    # extract group with tags from element tuple
+    _, group = element
+
+    existing_person = person_state.read()
+    if existing_person:
+      # the person exists in person_state for this person id
+      for auction in group[nexmark_query_util.AUCTION_TAG]:
+        yield auction, existing_person
+      return
+
+    new_person = None
+    for person in group[nexmark_query_util.PERSON_TAG]:
+      if not new_person:
+        new_person = person
+      else:
+        logging.error(
+            'two new person wtih same key: %s and %s' % (person, new_person))
+        continue
+      # read all pending auctions for this person id, output and flush it
+      pending_auctions = auction_state.read()
+      if pending_auctions:
+        for pending_auction in pending_auctions:
+          yield pending_auction, new_person
+        auction_state.clear()
+      # output new auction for this person id
+      for auction in group[nexmark_query_util.AUCTION_TAG]:
+        yield auction, new_person
+      # remember person for max_auction_wait_time seconds for future auctions
+      person_state.write(new_person)
+      person_timer.set(new_person.date_time + self.max_auction_wait_time)
+    # we are done if we have seen a new person
+    if new_person:
+      return
+
+    # remember auction until we see person
+    for auction in group[nexmark_query_util.AUCTION_TAG]:
+      auction_state.add(auction)
+
+  @on_timer(person_timer_spec)
+  def expiry(self, person_state=beam.DoFn.StateParam(person_spec)):
+    person_state.clear()

Review comment:
       Can it happen that the same person creates new auctions after > 
`max_auction_waiting_time` ? Will there be a new person event? If we get new 
auctions after the person is expired, then we'll just keep adding them to 
auction_state forevetr, no?

##########
File path: 
sdks/python/apache_beam/testing/benchmarks/nexmark/nexmark_launcher.py
##########
@@ -262,13 +285,13 @@ def run(self):
       query_duration = 
self.pipeline_options.view_as(TestOptions).wait_until_finish_duration  # 
pylint: disable=line-too-long
       if launch_from_direct_runner:
         command = Command(
-            self.run_query, args=[queries[i], query_args.get(i), query_errors])
+            self.run_query, args=[queries[i], query_args, query_errors])
         command.run(timeout=query_duration // 1000)
       else:
         try:
-          self.run_query(queries[i], query_args.get(i), query_errors=None)
+          self.run_query(queries[i], query_args, query_errors=query_errors)

Review comment:
       how come we don't have per-query query-args anymore?

##########
File path: sdks/python/apache_beam/testing/benchmarks/nexmark/queries/query6.py
##########
@@ -0,0 +1,92 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+"""
+Query 6, 'Average Selling Price by Seller'. Select the average selling price
+over the last 10 closed auctions by the same seller. In CQL syntax::
+
+  SELECT Istream(AVG(Q.final), Q.seller)
+  FROM (SELECT Rstream(MAX(B.price) AS final, A.seller)
+    FROM Auction A [ROWS UNBOUNDED], Bid B [ROWS UNBOUNDED]
+    WHERE A.id=B.auction
+      AND B.datetime < A.expires AND A.expires < CURRENT_TIME
+    GROUP BY A.id, A.seller) [PARTITION BY A.seller ROWS 10] Q
+  GROUP BY Q.seller;
+"""
+
+from __future__ import absolute_import
+from __future__ import division
+
+import apache_beam as beam
+from apache_beam.testing.benchmarks.nexmark.queries import nexmark_query_util
+from apache_beam.testing.benchmarks.nexmark.queries import winning_bids
+from apache_beam.testing.benchmarks.nexmark.queries.nexmark_query_util import 
ResultNames
+from apache_beam.transforms import trigger
+from apache_beam.transforms import window
+
+
+def load(events, metadata=None):
+  # find winning bids for each closed auction
+  return (
+      events
+      # find winning bids
+      | beam.Filter(nexmark_query_util.auction_or_bid)
+      | winning_bids.WinningBids()
+      # (auction_bids -> (aution.seller, bid)
+      | beam.Map(lambda auc_bid: (auc_bid.auction.seller, auc_bid.bid))
+      # calculate and output mean as data arrives
+      | beam.WindowInto(
+          window.GlobalWindows(),
+          trigger=trigger.Repeatedly(trigger.AfterCount(1)),
+          accumulation_mode=trigger.AccumulationMode.ACCUMULATING,
+          allowed_lateness=0)
+      | beam.CombinePerKey(MovingMeanSellingPriceFn(10))
+      | beam.Map(lambda t: {
+          ResultNames.SELLER: t[0], ResultNames.PRICE: t[1]
+      }))
+
+
+class MovingMeanSellingPriceFn(beam.CombineFn):
+  """
+  Combiner to keep track of up to max_num_bids of the most recent wining
+  bids and calculate their average selling price.
+  """
+  def __init__(self, max_num_bids):
+    self.max_num_bids = max_num_bids
+
+  def create_accumulator(self):
+    return []
+
+  def add_input(self, accumulator, element):
+    accumulator.append(element)
+    new_accu = sorted(accumulator, key=lambda bid: (bid.date_time, bid.price))
+    if len(new_accu) > self.max_num_bids:
+      del new_accu[0]

Review comment:
       Deleting from the begining of a Python list is very slow (as all 
elements have to be shifted). Should the sorting be opposite and wwe just drop 
from the end?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to