y1chi commented on a change in pull request #12580: URL: https://github.com/apache/beam/pull/12580#discussion_r470710383
########## File path: sdks/python/apache_beam/testing/benchmarks/nexmark/models/bids_per_session.py ########## @@ -0,0 +1,56 @@ +# +# 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. +# + +"""Result of Query11 and 12.""" +from __future__ import absolute_import + +from apache_beam.coders import coder_impl +from apache_beam.coders.coders import FastCoder +from apache_beam.testing.benchmarks.nexmark import nexmark_util + + +class BidsPerSessionCoder(FastCoder): Review comment: if there is no need to serialize/deserialize the object to bytes and send it over a network channel then the coder is typically not needed. The result classes seem only used to print result and there seem no additional transforms that require further processing of these objects. I think we could avoid having these result classes along with their coders, I don't think they add too much value to the pipeline. ########## File path: sdks/python/apache_beam/testing/benchmarks/nexmark/queries/query3.py ########## @@ -0,0 +1,164 @@ +# +# 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.coders import coders +from apache_beam.testing.benchmarks.nexmark.models import name_city_state_id +from apache_beam.testing.benchmarks.nexmark.models import nexmark_model +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 +from apache_beam.utils.timestamp import Duration + + +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, + allowed_lateness=Duration.of(0))) + 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: name_city_state_id.NameCiyStateId( + t[1].name, t[1].city, t[1].state, 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.ReadModifyWriteStateSpec( Review comment: not necessarily, ReadModifyWriteState uses BagUserState of size 1 underneath, if you use ReadModifyWriteState the whole list gets serialized/deserialized each time as a single item you make a change, BagUserState allows you to just add the new items. ########## File path: sdks/python/apache_beam/testing/benchmarks/nexmark/queries/query6.py ########## @@ -0,0 +1,93 @@ +# +# 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.models import seller_price +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 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.combiners.Count.Globally()) + | beam.CombinePerKey(MovingMeanSellingPriceFn(10)) + | beam.Map(lambda t: seller_price.SellerPrice(t[0], 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] + return new_accu + + def merge_accumulators(self, accumulators): + new_accu = [] + for accumulator in accumulators: + new_accu += accumulator + new_accu.sort(key=lambda bid: (bid.date_time, bid.price)) + return new_accu[-10:] + + def extract_output(self, accumulator): + if len(accumulator) == 0: + return 0 + sum_price = 0 + for bid in accumulator: Review comment: yeah. ---------------------------------------------------------------- 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]
