[
https://issues.apache.org/jira/browse/BEAM-2855?focusedWorklogId=117084&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-117084
]
ASF GitHub Bot logged work on BEAM-2855:
----------------------------------------
Author: ASF GitHub Bot
Created on: 28/Jun/18 22:22
Start Date: 28/Jun/18 22:22
Worklog Time Spent: 10m
Work Description: aaltay closed pull request #5738: [BEAM-2855] Add
Python Nexmark suite
URL: https://github.com/apache/beam/pull/5738
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/__init__.py
b/sdks/python/apache_beam/testing/benchmarks/__init__.py
new file mode 100644
index 00000000000..f4f43cbb123
--- /dev/null
+++ b/sdks/python/apache_beam/testing/benchmarks/__init__.py
@@ -0,0 +1,17 @@
+#
+# 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.
+#
+from __future__ import absolute_import
diff --git a/sdks/python/apache_beam/testing/benchmarks/nexmark/__init__.py
b/sdks/python/apache_beam/testing/benchmarks/nexmark/__init__.py
new file mode 100644
index 00000000000..f4f43cbb123
--- /dev/null
+++ b/sdks/python/apache_beam/testing/benchmarks/nexmark/__init__.py
@@ -0,0 +1,17 @@
+#
+# 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.
+#
+from __future__ import absolute_import
diff --git
a/sdks/python/apache_beam/testing/benchmarks/nexmark/models/__init__.py
b/sdks/python/apache_beam/testing/benchmarks/nexmark/models/__init__.py
new file mode 100644
index 00000000000..f4f43cbb123
--- /dev/null
+++ b/sdks/python/apache_beam/testing/benchmarks/nexmark/models/__init__.py
@@ -0,0 +1,17 @@
+#
+# 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.
+#
+from __future__ import absolute_import
diff --git
a/sdks/python/apache_beam/testing/benchmarks/nexmark/models/nexmark_model.py
b/sdks/python/apache_beam/testing/benchmarks/nexmark/models/nexmark_model.py
new file mode 100644
index 00000000000..261cd502267
--- /dev/null
+++ b/sdks/python/apache_beam/testing/benchmarks/nexmark/models/nexmark_model.py
@@ -0,0 +1,85 @@
+#
+# 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 model.
+
+The nexmark suite is a series of queries (streaming pipelines) performed
+on a simulation of auction events. The model includes the three roles that
+generate events:
+
+ - The person who starts and auction or makes a bid (Person).
+ - The auction item (Auction).
+ - The bid on an item for auction (Bid).
+
+"""
+
+
+class Person(object):
+ "Author of an auction or a bid."
+
+ def __init__(self, id, name, email, credit_card,
+ city, state, timestamp, extra=None):
+ self.id = id
+ self.name = name
+ self.email = email # key
+ self.credit_card = credit_card
+ self.city = city
+ self.state = state
+ self.timestamp = timestamp
+ self.extra = extra
+
+ def __repr__(self):
+ return 'Person({id}, {email})'.format(**{'id': self.id,
+ 'email': self.email})
+
+
+class Auction(object):
+ "Item for auction."
+
+ def __init__(self, id, item_name, description, initial_bid, reserve_price,
+ timestamp, expires, seller, category, extra=None):
+ self.id = id
+ self.item_name = item_name # key
+ self.description = description
+ self.initial_bid = initial_bid
+ self.reserve_price = reserve_price
+ self.timestamp = timestamp
+ self.expires = expires
+ self.seller = seller
+ self.category = category
+ self.extra = extra
+
+ def __repr__(self):
+ return 'Auction({id}, {item_name})'.format(**{'id': self.id,
+ 'item_name': self.item_name})
+
+
+class Bid(object):
+ "A bid for an item for auction."
+
+ def __init__(self, auction, bidder, price, timestamp, extra=None):
+ self.auction = auction # key
+ self.bidder = bidder
+ self.price = price
+ self.timestamp = timestamp
+ self.extra = extra
+
+ def __repr__(self):
+ return 'Bid({auction}, {bidder}, {price})'.format(
+ **{'auction': self.auction,
+ 'bidder': self.bidder,
+ 'price': self.price})
diff --git
a/sdks/python/apache_beam/testing/benchmarks/nexmark/nexmark_launcher.py
b/sdks/python/apache_beam/testing/benchmarks/nexmark/nexmark_launcher.py
new file mode 100644
index 00000000000..3616f35c798
--- /dev/null
+++ b/sdks/python/apache_beam/testing/benchmarks/nexmark/nexmark_launcher.py
@@ -0,0 +1,246 @@
+#
+# 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> \
+ --loglevel=DEBUG (optional) \
+ --wait_until_finish_duration <time_in_ms> \
+ --streaming
+
+ - DataflowRunner
+ python nexmark_launcher.py \
+ --query/q <query number> \
+ --project <project id> \
+ --loglevel=DEBUG (optional) \
+ --wait_until_finish_duration <time_in_ms> \
+ --streaming \
+ --sdk_location <apache_beam tar.gz> \
+ --staging_location=gs://... \
+ --temp_location=gs://
+
+"""
+
+from __future__ import absolute_import
+from __future__ import print_function
+
+import argparse
+import logging
+import sys
+import uuid
+
+from google.cloud import pubsub
+
+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 apache_beam.testing.benchmarks.nexmark.nexmark_util import Command
+from apache_beam.testing.benchmarks.nexmark.queries import query0
+
+
+class NexmarkLauncher(object):
+ def __init__(self):
+ self.parse_args()
+ self.uuid = str(uuid.uuid4())
+ self.topic_name = self.args.topic_name + self.uuid
+ self.subscription_name = self.args.subscription_name + self.uuid
+
+ def parse_args(self):
+ parser = argparse.ArgumentParser()
+
+ parser.add_argument('--query', '-q',
+ type=int,
+ action='append',
+ required=True,
+ choices=[0, 1, 2],
+ help='Query to run')
+
+ parser.add_argument('--subscription_name',
+ type=str,
+ help='Pub/Sub subscription to read from')
+
+ parser.add_argument('--topic_name',
+ type=str,
+ help='Pub/Sub topic to read from')
+
+ parser.add_argument('--loglevel',
+ choices=['DEBUG', 'INFO', 'WARNING',
+ 'ERROR', 'CRITICAL'],
+ default='INFO',
+ help='Set logging level to debug')
+ parser.add_argument('--input',
+ type=str,
+ required=True,
+ help='Path to the data file containing nexmark
events.')
+
+ self.args, self.pipeline_args = parser.parse_known_args()
+ logging.basicConfig(level=getattr(logging, self.args.loglevel, None),
+ format='(%(threadName)-10s) %(message)s')
+
+ self.pipeline_options = PipelineOptions(self.pipeline_args)
+ logging.debug('args, pipeline_args: %s, %s', self.args, self.pipeline_args)
+
+ # Usage with Dataflow requires a project to be supplied.
+ self.project = self.pipeline_options.view_as(GoogleCloudOptions).project
+ if self.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.
+ self.streaming = self.pipeline_options.view_as(StandardOptions).streaming
+ if self.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.
+ self.wait_until_finish_duration = (
+ self.pipeline_options.view_as(TestOptions).wait_until_finish_duration
+ )
+ if self.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).
+ self.pipeline_options.view_as(SetupOptions).save_main_session = True
+
+ 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)
+
+ if self.args.input.startswith('gs://'):
+ from apache_beam.io.gcp.gcsfilesystem import GCSFileSystem
+ fs = GCSFileSystem(self.pipeline_options)
+ with fs.open(self.args.input) as infile:
+ for line in infile:
+ topic.publish(line)
+ else:
+ with open(self.args.input) as infile:
+ for line in infile:
+ topic.publish(line)
+
+ logging.info('Finished event generation.')
+
+ # Read from PubSub into a PCollection.
+ if self.args.subscription_name:
+ raw_events = self.pipeline | 'ReadPubSub' >> beam.io.ReadFromPubSub(
+ subscription=sub.full_name)
+ else:
+ raw_events = self.pipeline | 'ReadPubSub' >> beam.io.ReadFromPubSub(
+ topic=topic.full_name)
+
+ return raw_events
+
+ def run_query(self, query, query_errors):
+ try:
+ self.pipeline = beam.Pipeline(options=self.pipeline_options)
+ raw_events = self.generate_events()
+ query.load(raw_events)
+ result = self.pipeline.run()
+ job_duration = (
+ self.pipeline_options.view_as(TestOptions).wait_until_finish_duration
+ )
+ if self.pipeline_options.view_as(StandardOptions).runner ==
'DataflowRunner': # pylint: disable=line-too-long
+ result.wait_until_finish(duration=job_duration)
+ result.cancel()
+ else:
+ result.wait_until_finish()
+ except Exception as exc:
+ query_errors.append(str(exc))
+ raise
+
+ def cleanup(self):
+ publish_client = pubsub.Client(project=self.project)
+ topic = publish_client.topic(self.topic_name)
+ if topic.exists():
+ topic.delete()
+ sub = topic.subscription(self.subscription_name)
+ if sub.exists():
+ sub.delete()
+
+ def run(self):
+ queries = {
+ 0: query0,
+ # TODO(mariagh): Add more queries.
+ }
+
+ query_errors = []
+ for i in self.args.query:
+ self.parse_args()
+ logging.info('Running query %d', i)
+
+ # The DirectRunner is the default runner, and it needs
+ # special handling to cancel streaming jobs.
+ launch_from_direct_runner = self.pipeline_options.view_as(
+ StandardOptions).runner in [None, 'DirectRunner']
+
+ 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.run(timeout=query_duration // 1000)
+ else:
+ try:
+ self.run_query(queries[i], query_errors=None)
+ except Exception as exc:
+ query_errors.append(exc)
+
+ if query_errors:
+ logging.error('Query failed with %s', ', '.join(query_errors))
+ else:
+ logging.info('Queries run: %s', self.args.query)
+
+
+if __name__ == '__main__':
+ launcher = NexmarkLauncher()
+ launcher.run()
+ launcher.cleanup()
diff --git a/sdks/python/apache_beam/testing/benchmarks/nexmark/nexmark_util.py
b/sdks/python/apache_beam/testing/benchmarks/nexmark/nexmark_util.py
new file mode 100644
index 00000000000..dd8c142f97f
--- /dev/null
+++ b/sdks/python/apache_beam/testing/benchmarks/nexmark/nexmark_util.py
@@ -0,0 +1,99 @@
+#
+# 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.
+#
+
+"""Utilities for the Nexmark suite.
+
+The Nexmark suite is a series of queries (streaming pipelines) performed
+on a simulation of auction events. This util includes:
+
+ - A Command class used to terminate the streaming jobs
+ launched in nexmark_launcher.py by the DirectRunner.
+ - A ParseEventFn DoFn to parse events received from PubSub.
+
+Usage:
+
+To run a process for a certain duration, define in the code:
+ command = Command(process_to_terminate, args)
+ command.run(timeout=duration)
+
+"""
+
+from __future__ import absolute_import
+from __future__ import print_function
+
+import logging
+import threading
+
+import apache_beam as beam
+from apache_beam.testing.benchmarks.nexmark.models import nexmark_model
+
+
+class Command(object):
+ def __init__(self, cmd, args):
+ self.cmd = cmd
+ self.args = args
+ self.process = None
+
+ def run(self, timeout):
+ def thread_target():
+ logging.debug('Starting thread for %d seconds: %s',
+ timeout, self.cmd.__name__)
+
+ self.cmd(*self.args)
+ logging.info('%d seconds elapsed. Thread (%s) finished.',
+ timeout, self.cmd.__name__)
+
+ thread = threading.Thread(target=thread_target, name='Thread-timeout')
+ thread.daemon = True
+ thread.start()
+ thread.join(timeout)
+
+
+class ParseEventFn(beam.DoFn):
+ """Parses the raw event info into a Python objects.
+
+ Each event line has the following format:
+
+ person: <id starting with 'p'>,name,email,credit_card,city, \
+ state,timestamp,extra
+ auction: <id starting with 'a'>,item_name, description,initial_bid, \
+ reserve_price,timestamp,expires,seller,category,extra
+ bid: <auction starting with 'b'>,bidder,price,timestamp,extra
+
+ For example:
+
+ 'p12345,maria,[email protected],1234-5678-9012-3456, \
+ sunnyvale,CA,1528098831536'
+ 'a12345,car67,2012 hyundai elantra,15000,20000, \
+ 1528098831536,20180630,maria,vehicle'
+ 'b12345,maria,20000,1528098831536'
+ """
+
+ def process(self, elem):
+ model_dict = {
+ 'p': nexmark_model.Person,
+ 'a': nexmark_model.Auction,
+ 'b': nexmark_model.Bid,
+ }
+ row = elem.split(',')
+ model = model_dict.get(elem[0])
+ if not model:
+ raise ValueError('Invalid event: %s.' % row)
+
+ event = model(*row)
+ logging.debug('Parsed event: %s', event)
+ yield event
diff --git
a/sdks/python/apache_beam/testing/benchmarks/nexmark/queries/__init__.py
b/sdks/python/apache_beam/testing/benchmarks/nexmark/queries/__init__.py
new file mode 100644
index 00000000000..cce3acad34a
--- /dev/null
+++ b/sdks/python/apache_beam/testing/benchmarks/nexmark/queries/__init__.py
@@ -0,0 +1,16 @@
+#
+# 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.
+#
diff --git
a/sdks/python/apache_beam/testing/benchmarks/nexmark/queries/query0.py
b/sdks/python/apache_beam/testing/benchmarks/nexmark/queries/query0.py
new file mode 100644
index 00000000000..d26489fc9fa
--- /dev/null
+++ b/sdks/python/apache_beam/testing/benchmarks/nexmark/queries/query0.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.
+#
+
+"""Nexmark Query 0: Pass through (send and receive auction events).
+
+The Nexmark suite is a series of queries (streaming pipelines) performed
+on a simulation of auction events.
+
+This query is a pass through that
+simply parses the events generated by the launcher. It serves as a test
+to verify the infrastructure.
+"""
+
+from __future__ import absolute_import
+
+import apache_beam as beam
+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
----------------------------------------------------------------
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: 117084)
Time Spent: 12h (was: 11h 50m)
> 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: 12h
> 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)