You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by dh...@apache.org on 2016/08/15 21:35:30 UTC

incubator-beam git commit: [BEAM-378] integrate setuptools in Maven build

Repository: incubator-beam
Updated Branches:
  refs/heads/python-sdk 18ce31c1d -> 76de6109f


[BEAM-378] integrate setuptools in Maven build

This PR provide an initial integration of the Python SDK in the Maven
build, relying on the exec-maven-plugin to call setuptools (BEAM-378).


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/76de6109
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/76de6109
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/76de6109

Branch: refs/heads/python-sdk
Commit: 76de6109fa19db34390edc593a62524a92f54ca0
Parents: 18ce31c
Author: Sergio Fern�ndez <se...@wikier.org>
Authored: Mon Aug 15 14:34:42 2016 -0700
Committer: Dan Halperin <dh...@google.com>
Committed: Mon Aug 15 14:35:22 2016 -0700

----------------------------------------------------------------------
 pom.xml                                         |   3 +-
 sdks/pom.xml                                    |   4 +-
 sdks/python/MANIFEST.in                         |  19 +++
 sdks/python/apache_beam/io/fileio.py            |  25 +--
 sdks/python/apache_beam/io/gcsio.py             |   9 +-
 sdks/python/apache_beam/io/gcsio_test.py        |  23 +--
 sdks/python/apache_beam/runners/common.pxd      |   2 +-
 sdks/python/apache_beam/runners/common.py       |  18 +-
 .../python/apache_beam/runners/direct_runner.py |   9 +-
 .../runners/inprocess/transform_evaluator.py    |   8 +-
 .../apache_beam/transforms/aggregator_test.py   |   8 +-
 sdks/python/apache_beam/utils/dependency.py     |   1 -
 .../python/apache_beam/utils/dependency_test.py |   1 -
 sdks/python/apache_beam/version.py              |  42 ++++-
 sdks/python/pom.xml                             | 169 +++++++++++++++++++
 sdks/python/setup.cfg                           |   2 +
 sdks/python/setup.py                            |   2 +-
 sdks/python/tox.ini                             |   3 +
 18 files changed, 266 insertions(+), 82 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/76de6109/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 9e58ffe..afe24ee 100644
--- a/pom.xml
+++ b/pom.xml
@@ -778,7 +778,7 @@
         <plugin>
           <groupId>org.apache.rat</groupId>
           <artifactId>apache-rat-plugin</artifactId>
-          <version>0.11</version>
+          <version>0.12</version>
           <configuration>
             <reportFile>${project.build.directory}/${project.build.finalName}.rat</reportFile>
             <excludeSubProjects>false</excludeSubProjects>
@@ -794,6 +794,7 @@
               <exclude>**/test/resources/**/*.txt</exclude>
               <exclude>**/test/**/.placeholder</exclude>
               <exclude>.repository/**/*</exclude>
+              <exclude>**/nose-*.egg/**/*</exclude>
 
               <!-- Default eclipse excludes neglect subprojects -->
               <exclude>**/.checkstyle</exclude>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/76de6109/sdks/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/pom.xml b/sdks/pom.xml
index aa9cbed..fe37e96 100644
--- a/sdks/pom.xml
+++ b/sdks/pom.xml
@@ -34,6 +34,7 @@
 
   <modules>
     <module>java</module>
+    <module>python</module>
   </modules>
 
   <build>
@@ -53,8 +54,9 @@
             </execution>
           </executions>
         </plugin>
+
       </plugins>
     </pluginManagement>
   </build>
 
-</project>
\ No newline at end of file
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/76de6109/sdks/python/MANIFEST.in
----------------------------------------------------------------------
diff --git a/sdks/python/MANIFEST.in b/sdks/python/MANIFEST.in
new file mode 100644
index 0000000..baa2fda
--- /dev/null
+++ b/sdks/python/MANIFEST.in
@@ -0,0 +1,19 @@
+#
+# 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.
+#
+
+# This file is used from Python to sync versions
+include pom.xml

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/76de6109/sdks/python/apache_beam/io/fileio.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/io/fileio.py b/sdks/python/apache_beam/io/fileio.py
index a6ce26a..b1e091b 100644
--- a/sdks/python/apache_beam/io/fileio.py
+++ b/sdks/python/apache_beam/io/fileio.py
@@ -283,17 +283,7 @@ class _CompressionType(object):
     self.identifier = identifier
 
   def __eq__(self, other):
-    return (isinstance(other, _CompressionType) and
-            self.identifier == other.identifier)
-
-  def __hash__(self):
-    return hash(self.identifier)
-
-  def __ne__(self, other):
-    return not self.__eq__(other)
-
-  def __repr__(self):
-    return '_CompressionType(%s)' % self.identifier
+    return self.identifier == other.identifier
 
 
 class CompressionTypes(object):
@@ -540,22 +530,15 @@ class FileSink(iobase.Sink):
         channel_factory.rename(old_name, final_name)
       except IOError as e:
         # May have already been copied.
-        try:
-          exists = channel_factory.exists(final_name)
-        except Exception as exists_e:  # pylint: disable=broad-except
-          logging.warning('Exception when checking if file %s exists: '
-                          '%s', final_name, exists_e)
-          # Returning original exception after logging the exception from
-          # exists() call.
-          return (None, e)
+        exists = channel_factory.exists(final_name)
         if not exists:
           logging.warning(('IOError in _rename_file. old_name: %s, '
                            'final_name: %s, err: %s'), old_name, final_name, e)
-          return (None, e)
+          return(None, e)
       except Exception as e:  # pylint: disable=broad-except
         logging.warning(('Exception in _rename_file. old_name: %s, '
                          'final_name: %s, err: %s'), old_name, final_name, e)
-        return (None, e)
+        return(None, e)
       return (final_name, None)
 
     # ThreadPool crashes in old versions of Python (< 2.7.5) if created from a

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/76de6109/sdks/python/apache_beam/io/gcsio.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/io/gcsio.py b/sdks/python/apache_beam/io/gcsio.py
index 7bb532c..88fcfb8 100644
--- a/sdks/python/apache_beam/io/gcsio.py
+++ b/sdks/python/apache_beam/io/gcsio.py
@@ -234,13 +234,8 @@ class GcsIO(object):
                                                  object=object_path)
       self.client.objects.Get(request)  # metadata
       return True
-    except HttpError as http_error:
-      if http_error.status_code == 404:
-        # HTTP 404 indicates that the file did not exist
-        return False
-      else:
-        # We re-raise all other exceptions
-        raise
+    except IOError:
+      return False
 
   @retry.with_exponential_backoff(
       retry_filter=retry.retry_on_server_errors_and_timeout_filter)

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/76de6109/sdks/python/apache_beam/io/gcsio_test.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/io/gcsio_test.py b/sdks/python/apache_beam/io/gcsio_test.py
index 99c99b3..7b15ef3 100644
--- a/sdks/python/apache_beam/io/gcsio_test.py
+++ b/sdks/python/apache_beam/io/gcsio_test.py
@@ -29,7 +29,6 @@ from apitools.base.py.exceptions import HttpError
 from apache_beam.internal.clients import storage
 
 from apache_beam.io import gcsio
-from mock import patch
 
 
 class FakeGcsClient(object):
@@ -80,8 +79,7 @@ class FakeGcsObjects(object):
   def Get(self, get_request, download=None):  # pylint: disable=invalid-name
     f = self.get_file(get_request.bucket, get_request.object)
     if f is None:
-      # Failing with a HTTP 404 if file does not exist.
-      raise HttpError({'status':404}, None, None)
+      raise ValueError('Specified object does not exist.')
     if download is None:
       return f.get_metadata()
     else:
@@ -191,25 +189,6 @@ class TestGCSIO(unittest.TestCase):
     self.client = FakeGcsClient()
     self.gcs = gcsio.GcsIO(self.client)
 
-  def test_exists(self):
-    file_name = 'gs://gcsio-test/dummy_file'
-    file_size = 1234
-    self._insert_random_file(self.client, file_name, file_size)
-    self.assertFalse(self.gcs.exists(file_name + 'xyz'))
-    self.assertTrue(self.gcs.exists(file_name))
-
-  @patch.object(FakeGcsObjects, 'Get')
-  def test_exists_failure(self, mock_get):
-    # Raising an error other than 404. Raising 404 is a valid failure for
-    # exists() call.
-    mock_get.side_effect = HttpError({'status':400}, None, None)
-    file_name = 'gs://gcsio-test/dummy_file'
-    file_size = 1234
-    self._insert_random_file(self.client, file_name, file_size)
-    with self.assertRaises(HttpError) as cm:
-      self.gcs.exists(file_name)
-    self.assertEquals(400, cm.exception.status_code)
-
   def test_size(self):
     file_name = 'gs://gcsio-test/dummy_file'
     file_size = 1234

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/76de6109/sdks/python/apache_beam/runners/common.pxd
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/runners/common.pxd b/sdks/python/apache_beam/runners/common.pxd
index 5cd4cf8..7acd049 100644
--- a/sdks/python/apache_beam/runners/common.pxd
+++ b/sdks/python/apache_beam/runners/common.pxd
@@ -32,7 +32,7 @@ cdef class DoFnRunner(Receiver):
   cdef object dofn
   cdef object dofn_process
   cdef object window_fn
-  cdef DoFnContext context
+  cdef object context   # TODO(robertwb): Make this a DoFnContext
   cdef object tagged_receivers
   cdef LoggingContext logging_context
   cdef object step_name

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/76de6109/sdks/python/apache_beam/runners/common.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/runners/common.py b/sdks/python/apache_beam/runners/common.py
index 67277c3..c017704 100644
--- a/sdks/python/apache_beam/runners/common.py
+++ b/sdks/python/apache_beam/runners/common.py
@@ -59,16 +59,12 @@ class DoFnRunner(Receiver):
                kwargs,
                side_inputs,
                windowing,
-               context=None,
-               tagged_receivers=None,
+               context,
+               tagged_receivers,
                logger=None,
                step_name=None,
                # Preferred alternative to logger
-               # TODO(robertwb): Remove once all runners are updated.
-               logging_context=None,
-               # Preferred alternative to context
-               # TODO(robertwb): Remove once all runners are updated.
-               state=None):
+               logging_context=None):
     if not args and not kwargs:
       self.dofn = fn
       self.dofn_process = fn.process
@@ -89,16 +85,10 @@ class DoFnRunner(Receiver):
       self.dofn_process = lambda context: fn.process(context, *args, **kwargs)
 
     self.window_fn = windowing.windowfn
+    self.context = context
     self.tagged_receivers = tagged_receivers
     self.step_name = step_name
 
-    if state:
-      assert context is None
-      self.context = DoFnContext(self.step_name, state=state)
-    else:
-      assert context is not None
-      self.context = context
-
     if logging_context:
       self.logging_context = logging_context
     else:

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/76de6109/sdks/python/apache_beam/runners/direct_runner.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/runners/direct_runner.py b/sdks/python/apache_beam/runners/direct_runner.py
index a62ddf7..e0df439 100644
--- a/sdks/python/apache_beam/runners/direct_runner.py
+++ b/sdks/python/apache_beam/runners/direct_runner.py
@@ -43,6 +43,7 @@ from apache_beam.runners.runner import PipelineResult
 from apache_beam.runners.runner import PipelineRunner
 from apache_beam.runners.runner import PipelineState
 from apache_beam.runners.runner import PValueCache
+from apache_beam.transforms import DoFnProcessContext
 from apache_beam.transforms.window import GlobalWindows
 from apache_beam.transforms.window import WindowedValue
 from apache_beam.typehints.typecheck import OutputCheckWrapperDoFn
@@ -137,6 +138,9 @@ class DirectPipelineRunner(PipelineRunner):
   @skip_if_cached
   def run_ParDo(self, transform_node):
     transform = transform_node.transform
+    # TODO(gildea): what is the appropriate object to attach the state to?
+    context = DoFnProcessContext(label=transform.label,
+                                 state=DoFnState(self._counter_factory))
 
     side_inputs = [self._cache.get_pvalue(view)
                    for view in transform_node.side_inputs]
@@ -172,9 +176,8 @@ class DirectPipelineRunner(PipelineRunner):
 
     runner = DoFnRunner(transform.dofn, transform.args, transform.kwargs,
                         side_inputs, transform_node.inputs[0].windowing,
-                        tagged_receivers=TaggedReceivers(),
-                        step_name=transform_node.full_label,
-                        state=DoFnState(self._counter_factory))
+                        context, TaggedReceivers(),
+                        step_name=transform_node.full_label)
     runner.start()
     for v in self._cache.get_pvalue(transform_node.inputs[0]):
       runner.process(v)

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/76de6109/sdks/python/apache_beam/runners/inprocess/transform_evaluator.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/runners/inprocess/transform_evaluator.py b/sdks/python/apache_beam/runners/inprocess/transform_evaluator.py
index 9aeda46..138ea87 100644
--- a/sdks/python/apache_beam/runners/inprocess/transform_evaluator.py
+++ b/sdks/python/apache_beam/runners/inprocess/transform_evaluator.py
@@ -30,6 +30,7 @@ from apache_beam.runners.common import DoFnState
 from apache_beam.runners.inprocess.inprocess_watermark_manager import InProcessWatermarkManager
 from apache_beam.runners.inprocess.inprocess_transform_result import InProcessTransformResult
 from apache_beam.transforms import core
+from apache_beam.transforms import DoFnProcessContext
 from apache_beam.transforms import sideinputs
 from apache_beam.transforms.window import GlobalWindows
 from apache_beam.transforms.window import WindowedValue
@@ -336,6 +337,8 @@ class _ParDoEvaluator(_TransformEvaluator):
     self._tagged_receivers[None].tag = None  # main_tag is None.
 
     self._counter_factory = counters.CounterFactory()
+    context = DoFnProcessContext(label=transform.label,
+                                 state=DoFnState(self._counter_factory))
 
     dofn = copy.deepcopy(transform.dofn)
 
@@ -348,9 +351,8 @@ class _ParDoEvaluator(_TransformEvaluator):
     self.runner = DoFnRunner(dofn, transform.args, transform.kwargs,
                              self._side_inputs,
                              self._applied_ptransform.inputs[0].windowing,
-                             tagged_receivers=self._tagged_receivers,
-                             step_name=self._applied_ptransform.full_label,
-                             state=DoFnState(self._counter_factory))
+                             context, self._tagged_receivers,
+                             step_name=self._applied_ptransform.full_label)
     self.runner.start()
 
   def process_element(self, element):

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/76de6109/sdks/python/apache_beam/transforms/aggregator_test.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/transforms/aggregator_test.py b/sdks/python/apache_beam/transforms/aggregator_test.py
index 040819e..0b3a20b 100644
--- a/sdks/python/apache_beam/transforms/aggregator_test.py
+++ b/sdks/python/apache_beam/transforms/aggregator_test.py
@@ -55,18 +55,18 @@ class AggregatorTest(unittest.TestCase):
         (any, int, True),
         (all, float, False),
     ]
-    aggregators = [Aggregator('%s_%s' % (f.__name__, t.__name__), f, t)
-                   for f, t, _ in counter_types]
+    aggeregators = [Aggregator('%s_%s' % (f.__name__, t.__name__), f, t)
+                    for f, t, _ in counter_types]
 
     class UpdateAggregators(beam.DoFn):
       def process(self, context):
-        for a in aggregators:
+        for a in aggeregators:
           context.aggregate_to(a, context.element)
 
     p = beam.Pipeline('DirectPipelineRunner')
     p | beam.Create([0, 1, 2, 3]) | beam.ParDo(UpdateAggregators())  # pylint: disable=expression-not-assigned
     res = p.run()
-    for (_, _, expected), a in zip(counter_types, aggregators):
+    for (_, _, expected), a in zip(counter_types, aggeregators):
       actual = res.aggregated_values(a).values()[0]
       self.assertEqual(expected, actual)
       self.assertEqual(type(expected), type(actual))

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/76de6109/sdks/python/apache_beam/utils/dependency.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/utils/dependency.py b/sdks/python/apache_beam/utils/dependency.py
index 7d1ae41..aacaf90 100644
--- a/sdks/python/apache_beam/utils/dependency.py
+++ b/sdks/python/apache_beam/utils/dependency.py
@@ -66,7 +66,6 @@ from apache_beam.utils import names
 from apache_beam.utils import processes
 from apache_beam.utils.options import GoogleCloudOptions
 from apache_beam.utils.options import SetupOptions
-from apache_beam.version import __version__
 
 
 # Standard file names used for staging files.

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/76de6109/sdks/python/apache_beam/utils/dependency_test.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/utils/dependency_test.py b/sdks/python/apache_beam/utils/dependency_test.py
index 2f9a57b..ca31806 100644
--- a/sdks/python/apache_beam/utils/dependency_test.py
+++ b/sdks/python/apache_beam/utils/dependency_test.py
@@ -29,7 +29,6 @@ from apache_beam.utils import names
 from apache_beam.utils.options import GoogleCloudOptions
 from apache_beam.utils.options import PipelineOptions
 from apache_beam.utils.options import SetupOptions
-from apache_beam.version import __version__
 
 
 class SetupTest(unittest.TestCase):

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/76de6109/sdks/python/apache_beam/version.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/version.py b/sdks/python/apache_beam/version.py
index c4c99f3..60d9634 100644
--- a/sdks/python/apache_beam/version.py
+++ b/sdks/python/apache_beam/version.py
@@ -15,6 +15,44 @@
 # limitations under the License.
 #
 
-"""Apache Beam SDK version information."""
+"""Apache Beam SDK version information and utilities."""
 
-__version__ = '0.3.0'
+
+import re
+
+
+__version__ = '0.3.0-incubating.dev'  # TODO: PEP 440 and incubating suffix
+
+
+# The following utilities are legacy code from the Maven integration;
+# see BEAM-378 for further details.
+
+
+# Reads the actual version from pom.xml file,
+def get_version_from_pom():
+  with open('pom.xml', 'r') as f:
+    pom = f.read()
+    regex = (r'.*<parent>\s*'
+             r'<groupId>[a-z\.]+</groupId>\s*'
+             r'<artifactId>[a-z\-]+</artifactId>\s*'
+             r'<version>([0-9a-zA-Z\.\-]+)</version>.*')
+    pattern = re.compile(str(regex))
+    search = pattern.search(pom)
+    version = search.group(1)
+    version = version.replace("-SNAPSHOT", ".dev")
+    # TODO: PEP 440 and incubating suffix
+    return version
+
+
+# Synchronizes apache_beam.__version__ field for later usage
+def sync_version(version):
+  init_path = 'apache_beam/__init__.py'
+  regex = r'^__version__\s*=\s*".*"'
+  with open(init_path, "r") as f:
+    lines = f.readlines()
+  with open(init_path, "w") as f:
+    for line in lines:
+      if re.search(regex, line):
+        f.write(re.sub(regex, '__version__ = "%s"' % version, line))
+      else:
+        f.write(line)

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/76de6109/sdks/python/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/python/pom.xml b/sdks/python/pom.xml
new file mode 100644
index 0000000..79d20bf
--- /dev/null
+++ b/sdks/python/pom.xml
@@ -0,0 +1,169 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+    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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+
+  <modelVersion>4.0.0</modelVersion>
+
+  <parent>
+    <groupId>org.apache.beam</groupId>
+    <artifactId>beam-sdks-parent</artifactId>
+    <version>0.3.0-incubating-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+
+  <artifactId>beam-sdks-python</artifactId>
+
+  <packaging>pom</packaging>
+
+  <name>Apache Beam :: SDKs :: Python</name>
+
+  <properties>
+    <python.interpreter.bin>python2</python.interpreter.bin>
+    <python.pip.bin>pip2</python.pip.bin>
+    <python.build.base>${project.build.directory}/build</python.build.base>
+    <python.user.base>${project.build.directory}/python</python.user.base>
+  </properties>
+
+  <build>
+    <pluginManagement>
+      <plugins>
+        <plugin>
+          <groupId>org.codehaus.mojo</groupId>
+          <artifactId>exec-maven-plugin</artifactId>
+          <executions>
+            <execution>
+              <!-- trick by @aaltay to skip inheritance -->
+              <phase>none</phase>
+            </execution>
+          </executions>
+        </plugin>
+      </plugins>
+    </pluginManagement>
+    <plugins>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>exec-maven-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>setuptools-clean</id>
+            <phase>clean</phase>
+            <goals>
+              <goal>exec</goal>
+            </goals>
+            <configuration>
+              <executable>${python.interpreter.bin}</executable>
+              <arguments>
+                <argument>setup.py</argument>
+                <argument>clean</argument>
+              </arguments>
+            </configuration>
+          </execution>
+          <execution>
+            <id>setuptools-build</id>
+            <phase>compile</phase>
+            <goals>
+              <goal>exec</goal>
+            </goals>
+            <configuration>
+              <executable>${python.interpreter.bin}</executable>
+              <arguments>
+                <argument>setup.py</argument>
+                <argument>build</argument>
+                <argument>--build-base</argument>
+                <argument>${python.build.base}</argument>
+              </arguments>
+            </configuration>
+          </execution>
+          <execution>
+            <id>setup-test-create-python-base</id>
+            <phase>process-test-resources</phase>
+            <goals>
+              <goal>exec</goal>
+            </goals>
+            <configuration>
+              <executable>mkdir</executable>
+              <arguments>
+                <argument>-p</argument>
+                <argument>${python.user.base}</argument>
+              </arguments>
+            </configuration>
+          </execution>
+          <execution>
+            <id>setup-test-tox</id>
+            <phase>process-test-resources</phase>
+            <goals>
+              <goal>exec</goal>
+            </goals>
+            <configuration>
+              <executable>${python.pip.bin}</executable>
+              <arguments>
+                <argument>install</argument>
+                <argument>--user</argument>
+                <argument>--upgrade</argument>
+                <argument>--ignore-installed</argument>
+                <argument>tox</argument>
+              </arguments>
+              <environmentVariables>
+                <PYTHONUSERBASE>${python.user.base}</PYTHONUSERBASE>
+              </environmentVariables>
+            </configuration>
+          </execution>
+          <execution>
+            <id>setuptools-test</id>
+            <phase>test</phase>
+            <goals>
+              <goal>exec</goal>
+            </goals>
+            <configuration>
+              <executable>${python.user.base}/bin/tox</executable>
+              <arguments>
+                <argument>-e</argument>
+                <argument>py27</argument>
+                <argument>-c</argument>
+                <argument>tox.ini</argument>
+              </arguments>
+              <environmentVariables>
+                <PYTHONUSERBASE>${python.user.base}</PYTHONUSERBASE>
+              </environmentVariables>
+            </configuration>
+          </execution>
+          <execution>
+            <id>setuptools-sdist</id>
+            <phase>package</phase>
+            <goals>
+              <goal>exec</goal>
+            </goals>
+            <configuration>
+              <executable>${python.interpreter.bin}</executable>
+              <arguments>
+                <argument>setup.py</argument>
+                <argument>sdist</argument>
+                <argument>--dist-dir</argument>
+                <argument>${project.build.directory}</argument>
+              </arguments>
+              <environmentVariables>
+                <PYTHONUSERBASE>${python.user.base}</PYTHONUSERBASE>
+              </environmentVariables>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/76de6109/sdks/python/setup.cfg
----------------------------------------------------------------------
diff --git a/sdks/python/setup.cfg b/sdks/python/setup.cfg
index 547a74b..fcfe003 100644
--- a/sdks/python/setup.cfg
+++ b/sdks/python/setup.cfg
@@ -26,3 +26,5 @@ verbosity=2
 # fast_coders_test and typecoders_test.
 exclude=fast_coders_test|typecoders_test
 
+[egg_info]
+egg_base = target

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/76de6109/sdks/python/setup.py
----------------------------------------------------------------------
diff --git a/sdks/python/setup.py b/sdks/python/setup.py
index b0a5c85..58e677e 100644
--- a/sdks/python/setup.py
+++ b/sdks/python/setup.py
@@ -98,6 +98,6 @@ setuptools.setup(
         'Topic :: Software Development :: Libraries',
         'Topic :: Software Development :: Libraries :: Python Modules',
         ],
-    license='Apache 2.0',
+    license='Apache License, Version 2.0',
     keywords=PACKAGE_KEYWORDS,
     )

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/76de6109/sdks/python/tox.ini
----------------------------------------------------------------------
diff --git a/sdks/python/tox.ini b/sdks/python/tox.ini
index 5a2572e..cba9626 100644
--- a/sdks/python/tox.ini
+++ b/sdks/python/tox.ini
@@ -17,6 +17,9 @@
 
 [tox]
 envlist = py27
+toxworkdir={toxinidir}/target/tox
+distdir={toxinidir}/target/dist
+distshare={toxinidir}/target/distshare
 
 [pep8]
 # Disable all errors and warnings except for the ones related to blank lines.