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

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

                Author: ASF GitHub Bot
            Created on: 09/Jul/18 17:46
            Start Date: 09/Jul/18 17:46
    Worklog Time Spent: 10m 
      Work Description: ryan-williams closed pull request #5902: [BEAM-4742] 
allow custom docker image in portable runner
URL: https://github.com/apache/beam/pull/5902
 
 
   

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/examples/wordcount.py 
b/sdks/python/apache_beam/examples/wordcount.py
index 3ba3b334188..8d0ddc1afdd 100644
--- a/sdks/python/apache_beam/examples/wordcount.py
+++ b/sdks/python/apache_beam/examples/wordcount.py
@@ -21,11 +21,13 @@
 
 import argparse
 import logging
+import os
 import re
 
 import apache_beam as beam
 from apache_beam.io import ReadFromText
 from apache_beam.io import WriteToText
+from apache_beam.io.filesystems import FileSystems
 from apache_beam.metrics import Metrics
 from apache_beam.metrics.metric import MetricsFilter
 from apache_beam.options.pipeline_options import PipelineOptions
@@ -111,6 +113,10 @@ def format_result(word_count):
 
   output = counts | 'format' >> beam.Map(format_result)
 
+  out_dir = os.path.dirname(known_args.output)
+  if not FileSystems.exists(out_dir):
+    FileSystems.mkdirs(out_dir)
+
   # Write the output using a "Write" transform that has side effects.
   # pylint: disable=expression-not-assigned
   output | 'write' >> WriteToText(known_args.output)
diff --git a/sdks/python/apache_beam/runners/portability/portable_runner.py 
b/sdks/python/apache_beam/runners/portability/portable_runner.py
index fff9aa49c17..be69fdf05ff 100644
--- a/sdks/python/apache_beam/runners/portability/portable_runner.py
+++ b/sdks/python/apache_beam/runners/portability/portable_runner.py
@@ -59,7 +59,15 @@ def __init__(self, is_embedded_fnapi_runner=False):
 
   @staticmethod
   def default_docker_image():
-    if 'USER' in os.environ:
+    if 'DOCKER_IMAGE' in os.environ:
+      # Perhaps also test if this was built?
+      image = os.environ['DOCKER_IMAGE'] + ':latest'
+      logging.info(
+          'Using latest locally built Python SDK docker image: %s',
+          image
+      )
+      return image
+    elif 'USER' in os.environ:
       # Perhaps also test if this was built?
       logging.info('Using latest locally built Python SDK docker image.')
       return os.environ['USER'] + 
'-docker-apache.bintray.io/beam/python:latest'


 

----------------------------------------------------------------
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: 120916)
    Time Spent: 1h 40m  (was: 1.5h)

> Allow custom docker-image in portable wordcount example
> -------------------------------------------------------
>
>                 Key: BEAM-4742
>                 URL: https://issues.apache.org/jira/browse/BEAM-4742
>             Project: Beam
>          Issue Type: Improvement
>          Components: examples-python
>    Affects Versions: 2.5.0
>            Reporter: Ryan Williams
>            Assignee: Ryan Williams
>            Priority: Minor
>          Time Spent: 1h 40m
>  Remaining Estimate: 0h
>
> I hit a couple snags [running the portable wordcount 
> example|https://github.com/apache/beam/blob/997ee3afe74483ae44e2dcb32ca0e24876129cd9/sdks/python/build.gradle#L200-L214]:
>  * -[the default docker image is hard-coded to a bintray 
> URL|https://github.com/apache/beam/blob/997ee3afe74483ae44e2dcb32ca0e24876129cd9/sdks/python/apache_beam/runners/portability/portable_runner.py#L60-L68],
>  but I published my image to Docker Hub- I missed that [there's already a 
> pipeline option for 
> this|https://github.com/apache/beam/pull/5902#discussion_r201071859]! Thanks 
> [~lcwik]
>  * the default output path is in a temporary directory that doesn't exist at 
> the time of the {{open}} call, so I got {{IOError: [Errno 2] No such file or 
> directory}} 
> I'll send a PR with fixes to each of these shortly.
> I've also not found where to observe output from successfully running the 
> example.



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

Reply via email to