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

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

                Author: ASF GitHub Bot
            Created on: 23/Jun/18 01:39
            Start Date: 23/Jun/18 01:39
    Worklog Time Spent: 10m 
      Work Description: aaltay commented on a change in pull request #5738: 
[BEAM-2855] Add Python Nexmark suite
URL: https://github.com/apache/beam/pull/5738#discussion_r197598008
 
 

 ##########
 File path: 
sdks/python/apache_beam/testing/benchmarks/nexmark/nexmark_launcher.py
 ##########
 @@ -0,0 +1,199 @@
+#
+# 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 launcher.
+
+The Nexmark suite is a series of queries (streaming pipelines) performed
+on a simulation of auction events. The launcher orchestrates the generation
+and parsing of streaming events and the running of queries.
+
+Model
+- Person: Author of an auction or a bid.
+- Auction: Item under auction.
+- Bid: A bid for an item under auction.
+
+Events
+ - Create Person
+ - Create Auction
+ - Create Bid
+
+Queries
+- Query0: Pass through (send and receive auction events).
+
+Usage
+- DirectRunner
+python nexmark_launcher.py \
+    --query/q <query number> \
+    --project <project id> \
+    --streaming \
+
+- DataflowRunner
+python nexmark_launcher.py \
+    --query/q <query number> \
+    --project <project id> \
+    --streaming \
+    --sdk_location <apache_beam tar.gz> \
+    --staging_location=gs://... \
+    --temp_location=gs://
+
+"""
+from __future__ import print_function
+
+import argparse
+import logging
+import sys
+
+import apache_beam as beam
+from apache_beam.options.pipeline_options import GoogleCloudOptions
+from apache_beam.options.pipeline_options import PipelineOptions
+from apache_beam.options.pipeline_options import SetupOptions
+from apache_beam.options.pipeline_options import StandardOptions
+from apache_beam.options.pipeline_options import TestOptions
+
+from nexmark_util import Command
+from queries import query0
+
+logging.basicConfig(level=logging.INFO,
+                    format='(%(threadName)-10s) %(message)s')
+
+# Reserved configuration to run these benchmark suite.
+PUBSUB_PROJECT = 'google.com:clouddfe'
+PUBSUB_SUBSCRIPTION = 'nexmark-benchmark'
+PUBSUB_TOPIC = 'nexmark-benchmark'
+
+# TODO(mariagh): Make comprehensive and large
+EVENT_COUNT_PER_TYPE = 3
+PERSON_EVENT = 
'p12345,maria,ma...@maria.com,1234-5678-9012-3456,sunnyvale,CA,1528098831536' # 
pylint: disable=line-too-long
+AUCTION_EVENT = 'a12345,car,2012 hyundai 
elantra,20K,15K,1528098831536,20180630,maria,vehicle' # pylint: 
disable=line-too-long
+BID_EVENT = 'b12345,maria,354,1528098831536'
+
+
+def parse_args():
+  parser = argparse.ArgumentParser()
+
+  parser.add_argument('--query', '-q',
+                      type=int,
+                      action='append',
+                      required=True,
+                      choices=[0, 1],
+                      help='Query to run')
+
+  parser.add_argument('--subscription',
+                      type=str,
+                      help='Pub/Sub subscription to read from')
+
+  parser.add_argument('--topic',
+                      type=str,
+                      help='Pub/Sub topic to read from')
+
+  args, pipeline_args = parser.parse_known_args()
+  options = PipelineOptions(pipeline_args)
+  print('args, pipeline_args:', args, pipeline_args)
+
+  # Usage with Dataflow requires a project to be supplied.
+  if options.view_as(GoogleCloudOptions).project is None:
+    parser.print_usage()
+    print(sys.argv[0] + ': error: argument --project is required')
+    sys.exit(1)
+
+  # Pub/Sub is currently available for use only in streaming pipelines.
+  if options.view_as(StandardOptions).streaming is None:
+    parser.print_usage()
+    print(sys.argv[0] + ': error: argument --streaming is required')
+    sys.exit(1)
+
+  # wait_until_finish ensures that the streaming job is canceled.
+  if options.view_as(TestOptions).wait_until_finish_duration is None:
+    parser.print_usage()
+    print(sys.argv[0] + ': error: argument --wait_until_finish_duration is 
required') # pylint: disable=line-too-long
+    sys.exit(1)
+
+  # We use the save_main_session option because one or more DoFn's in this
+  # workflow rely on global context (e.g., a module imported at module level).
+  options.view_as(SetupOptions).save_main_session = True
+  return args, options
+
+
+def generate_events(pipeline, args, pipeline_options):
+  from google.cloud import pubsub
+  publish_client = pubsub.Client(project=PUBSUB_PROJECT)
+  topic = publish_client.topic(PUBSUB_TOPIC)
+  sub = topic.subscription(PUBSUB_SUBSCRIPTION)
+  assert topic.exists()
+  assert sub.exists()
+
+  logging.info('Generating %d auction events to topic %s',
+               EVENT_COUNT_PER_TYPE*3, topic.name)
+  for _ in xrange(EVENT_COUNT_PER_TYPE):
+    topic.publish(PERSON_EVENT)
+    topic.publish(AUCTION_EVENT)
+    topic.publish(BID_EVENT)
+
+  logging.info('Finished event generation.')
+
+  # Read from PubSub into a PCollection.
+  if args.subscription:
+    raw_events = pipeline | 'ReadPubSub' >> beam.io.ReadFromPubSub(
+        subscription=args.subscription)
+  else:
+    raw_events = pipeline | 'ReadPubSub' >> beam.io.ReadFromPubSub(
+        topic=args.topic)
+
+  return raw_events
+
+
+def run_query(query, args, pipeline_options):
+  try:
+    pipeline = beam.Pipeline(options=pipeline_options)
+    raw_events = generate_events(pipeline, args, pipeline_options)
+    query.load(raw_events)
+    result = pipeline.run()
+    job_duration = 
pipeline_options.view_as(TestOptions).wait_until_finish_duration # pylint: 
disable=line-too-long
+    if pipeline_options.view_as(StandardOptions).runner == 'DataflowRunner':
+      result.wait_until_finish(duration=job_duration)
+      result.cancel()
+    else:
+      result.wait_until_finish()
+  except NotImplementedError as exp:
 
 Review comment:
   What would throw NotImplementedError in the above code?

----------------------------------------------------------------
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:
us...@infra.apache.org


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

    Worklog Id:     (was: 114996)
    Time Spent: 1h 50m  (was: 1h 40m)

> 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: 1h 50m
>  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