[ 
https://issues.apache.org/jira/browse/BEAM-2855?focusedWorklogId=120107&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-120107
 ]

ASF GitHub Bot logged work on BEAM-2855:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 06/Jul/18 22:32
            Start Date: 06/Jul/18 22:32
    Worklog Time Spent: 10m 
      Work Description: charlesccychen closed pull request #5837: [BEAM-2855] 
Add queries 1 and 2 to the Python Nexmark suite
URL: https://github.com/apache/beam/pull/5837
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/sdks/python/apache_beam/testing/benchmarks/nexmark/nexmark_launcher.py 
b/sdks/python/apache_beam/testing/benchmarks/nexmark/nexmark_launcher.py
index 3616f35c798..2779cb4fa5b 100644
--- a/sdks/python/apache_beam/testing/benchmarks/nexmark/nexmark_launcher.py
+++ b/sdks/python/apache_beam/testing/benchmarks/nexmark/nexmark_launcher.py
@@ -74,6 +74,8 @@
 from apache_beam.options.pipeline_options import TestOptions
 from apache_beam.testing.benchmarks.nexmark.nexmark_util import Command
 from apache_beam.testing.benchmarks.nexmark.queries import query0
+from apache_beam.testing.benchmarks.nexmark.queries import query1
+from apache_beam.testing.benchmarks.nexmark.queries import query2
 
 
 class NexmarkLauncher(object):
@@ -82,6 +84,19 @@ def __init__(self):
     self.uuid = str(uuid.uuid4())
     self.topic_name = self.args.topic_name + self.uuid
     self.subscription_name = self.args.subscription_name + self.uuid
+    publish_client = pubsub.Client(project=self.project)
+    topic = publish_client.topic(self.topic_name)
+    if topic.exists():
+      logging.info('deleting topic %s', self.topic_name)
+      topic.delete()
+    logging.info('creating topic %s', self.topic_name)
+    topic.create()
+    sub = topic.subscription(self.subscription_name)
+    if sub.exists():
+      logging.info('deleting sub %s', self.topic_name)
+      sub.delete()
+    logging.info('creating sub %s', self.topic_name)
+    sub.create()
 
   def parse_args(self):
     parser = argparse.ArgumentParser()
@@ -148,13 +163,7 @@ def parse_args(self):
   def generate_events(self):
     publish_client = pubsub.Client(project=self.project)
     topic = publish_client.topic(self.topic_name)
-    if topic.exists():
-      topic.delete()
-    topic.create()
     sub = topic.subscription(self.subscription_name)
-    if sub.exists():
-      sub.delete()
-    sub.create()
 
     logging.info('Generating auction events to topic %s', topic.name)
 
@@ -181,11 +190,12 @@ def generate_events(self):
 
     return raw_events
 
-  def run_query(self, query, query_errors):
+  def run_query(self, query, query_args, query_errors):
     try:
+      self.parse_args()
       self.pipeline = beam.Pipeline(options=self.pipeline_options)
       raw_events = self.generate_events()
-      query.load(raw_events)
+      query.load(raw_events, query_args)
       result = self.pipeline.run()
       job_duration = (
           self.pipeline_options.view_as(TestOptions).wait_until_finish_duration
@@ -211,9 +221,18 @@ def cleanup(self):
   def run(self):
     queries = {
         0: query0,
+        1: query1,
+        2: query2,
         # TODO(mariagh): Add more queries.
     }
 
+    # TODO(mariagh): Move to a config file.
+    query_args = {
+        2: {
+            'auction_id': 'a1003'
+        }
+    }
+
     query_errors = []
     for i in self.args.query:
       self.parse_args()
@@ -224,13 +243,15 @@ def run(self):
       launch_from_direct_runner = self.pipeline_options.view_as(
           StandardOptions).runner in [None, 'DirectRunner']
 
+      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_errors])
-        query_duration = 
self.pipeline_options.view_as(TestOptions).wait_until_finish_duration # pylint: 
disable=line-too-long
+        command = Command(self.run_query, args=[queries[i],
+                                                query_args.get(i),
+                                                query_errors])
         command.run(timeout=query_duration // 1000)
       else:
         try:
-          self.run_query(queries[i], query_errors=None)
+          self.run_query(queries[i], query_args.get(i), query_errors=None)
         except Exception as exc:
           query_errors.append(exc)
 
diff --git a/sdks/python/apache_beam/testing/benchmarks/nexmark/nexmark_util.py 
b/sdks/python/apache_beam/testing/benchmarks/nexmark/nexmark_util.py
index dd8c142f97f..688d60272cb 100644
--- a/sdks/python/apache_beam/testing/benchmarks/nexmark/nexmark_util.py
+++ b/sdks/python/apache_beam/testing/benchmarks/nexmark/nexmark_util.py
@@ -46,7 +46,6 @@ class Command(object):
   def __init__(self, cmd, args):
     self.cmd = cmd
     self.args = args
-    self.process = None
 
   def run(self, timeout):
     def thread_target():
@@ -97,3 +96,8 @@ def process(self, elem):
     event = model(*row)
     logging.debug('Parsed event: %s', event)
     yield event
+
+
+def display(elm):
+  logging.debug(elm)
+  return elm
diff --git 
a/sdks/python/apache_beam/testing/benchmarks/nexmark/queries/query0.py 
b/sdks/python/apache_beam/testing/benchmarks/nexmark/queries/query0.py
index d26489fc9fa..3df848d4eb9 100644
--- a/sdks/python/apache_beam/testing/benchmarks/nexmark/queries/query0.py
+++ b/sdks/python/apache_beam/testing/benchmarks/nexmark/queries/query0.py
@@ -31,5 +31,7 @@
 from apache_beam.testing.benchmarks.nexmark.nexmark_util import ParseEventFn
 
 
-def load(raw_events):
-  return raw_events | 'ParseEventFn' >> beam.ParDo(ParseEventFn())  # pylint: 
disable=expression-not-assigned
+def load(raw_events, query_args=None):
+  return (raw_events
+          | 'ParseEventFn' >> beam.ParDo(ParseEventFn())
+         )  # pylint: disable=expression-not-assigned
diff --git 
a/sdks/python/apache_beam/testing/benchmarks/nexmark/queries/query1.py 
b/sdks/python/apache_beam/testing/benchmarks/nexmark/queries/query1.py
new file mode 100644
index 00000000000..24b85796902
--- /dev/null
+++ b/sdks/python/apache_beam/testing/benchmarks/nexmark/queries/query1.py
@@ -0,0 +1,47 @@
+#
+# 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.
+#
+
+"""Nexmark Query 1: Convert bid prices from dollars to euros.
+
+The Nexmark suite is a series of queries (streaming pipelines) performed
+on a simulation of auction events.
+
+This query converts bid prices from dollars to euros.
+It illustrates a simple map.
+"""
+from __future__ import absolute_import
+
+import apache_beam as beam
+from apache_beam.testing.benchmarks.nexmark.models import nexmark_model
+from apache_beam.testing.benchmarks.nexmark.nexmark_util import ParseEventFn
+from apache_beam.testing.benchmarks.nexmark.nexmark_util import display
+
+
+def load(raw_events, query_args=None):
+  return (raw_events
+          | 'ParseEventFn' >> beam.ParDo(ParseEventFn())
+          | 'FilterInBids' >> beam.Filter(
+              lambda event: isinstance(event, nexmark_model.Bid))
+          | 'ConvertToEuro' >> beam.Map(
+              lambda bid: nexmark_model.Bid(
+                  bid.auction,
+                  bid.bidder,
+                  (float(bid.price) * 89) // 100,
+                  bid.timestamp,
+                  bid.extra))
+          | 'DisplayQuery1' >> beam.Map(display)
+         )  # pylint: disable=expression-not-assigned
diff --git 
a/sdks/python/apache_beam/testing/benchmarks/nexmark/queries/query2.py 
b/sdks/python/apache_beam/testing/benchmarks/nexmark/queries/query2.py
new file mode 100644
index 00000000000..33ee3f31c79
--- /dev/null
+++ b/sdks/python/apache_beam/testing/benchmarks/nexmark/queries/query2.py
@@ -0,0 +1,42 @@
+#
+# 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.
+#
+
+"""Nexmark Query 2: Select auctions by auction id.
+
+The Nexmark suite is a series of queries (streaming pipelines) performed
+on a simulation of auction events.
+
+This query selects auctions (items) that have a particular id.
+It illustrates a simple filter.
+"""
+
+from __future__ import absolute_import
+
+import apache_beam as beam
+from apache_beam.testing.benchmarks.nexmark.models import nexmark_model
+from apache_beam.testing.benchmarks.nexmark.nexmark_util import ParseEventFn
+from apache_beam.testing.benchmarks.nexmark.nexmark_util import display
+
+
+def load(raw_events, metadata=None):
+  return (raw_events
+          | 'ParseEventFn' >> beam.ParDo(ParseEventFn())
+          | 'FilterInAuctionsWithSelectedId' >> beam.Filter(
+              lambda event: (isinstance(event, nexmark_model.Auction)
+                             and event.id == metadata.get('auction_id')))
+          | 'DisplayQuery2' >> beam.Map(display)
+         )  # pylint: disable=expression-not-assigned


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


Issue Time Tracking
-------------------

    Worklog Id:     (was: 120107)
    Time Spent: 13h 40m  (was: 13.5h)

> Implement a python version of the nexmark queries
> -------------------------------------------------
>
>                 Key: BEAM-2855
>                 URL: https://issues.apache.org/jira/browse/BEAM-2855
>             Project: Beam
>          Issue Type: Improvement
>          Components: examples-nexmark, sdk-py-core, testing
>            Reporter: Ismaël Mejía
>            Assignee: María GH
>            Priority: Minor
>              Labels: newbie, nexmark, starter
>          Time Spent: 13h 40m
>  Remaining Estimate: 0h
>
> Currently we have a Java only implementation of Nexmark, a python based 
> implementation would be nice to have to validate the direct and dataflow 
> runners, but also to validate the new support of multiple SDKs in multiple 
> runners via the runner/fn API.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to