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



##########
File path: 
sdks/python/apache_beam/testing/benchmarks/nexmark/models/result_name.py
##########
@@ -0,0 +1,35 @@
+#
+# 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.
+#
+
+""" field names for the final results of queries
+"""
+
+
+class ResultNames:

Review comment:
       this can be put in nexmark_query_utils

##########
File path: sdks/python/apache_beam/testing/benchmarks/nexmark/queries/query3.py
##########
@@ -0,0 +1,160 @@
+#
+# 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.models.result_name import 
ResultNames
+from apache_beam.testing.benchmarks.nexmark.queries import nexmark_query_util
+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 == 'OR' or person.state == 'ID' or 
person.

Review comment:
       nit: `person.state in ['OR', 'ID', 'CA']`

##########
File path: sdks/python/apache_beam/testing/benchmarks/nexmark/queries/query4.py
##########
@@ -0,0 +1,80 @@
+#
+# 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.models.result_name import 
ResultNames
+from apache_beam.testing.benchmarks.nexmark.queries import nexmark_query_util
+from apache_beam.testing.benchmarks.nexmark.queries import winning_bids
+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 has bug, uncomment after it is fixed [BEAM-10617]

Review comment:
       fanout has bug -> fanout with sliding window produces duplicated results.

##########
File path: sdks/python/apache_beam/testing/benchmarks/nexmark/queries/query3.py
##########
@@ -0,0 +1,160 @@
+#
+# 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.models.result_name import 
ResultNames
+from apache_beam.testing.benchmarks.nexmark.queries import nexmark_query_util
+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 == 'OR' or person.state == 'ID' or 
person.
+          state == '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' %
+            (repr(person), repr(new_person)))

Review comment:
       'repr' seems redundant




----------------------------------------------------------------
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