You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by da...@apache.org on 2017/01/30 23:03:08 UTC

[01/50] [abbrv] beam git commit: Update DataflowPipelineResult.state at the end of poll_for_job_completion.

Repository: beam
Updated Branches:
  refs/heads/master 847e4e9f0 -> c3b97a287


Update DataflowPipelineResult.state at the end of poll_for_job_completion.


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

Branch: refs/heads/master
Commit: 56512ab442c599c64bfdb9fc6cabce95d76ee4dc
Parents: c03e6f3
Author: Ahmet Altay <al...@google.com>
Authored: Fri Jan 20 23:43:42 2017 -0800
Committer: Ahmet Altay <al...@google.com>
Committed: Fri Jan 20 23:43:42 2017 -0800

----------------------------------------------------------------------
 sdks/python/apache_beam/runners/dataflow_runner.py      | 6 ++++--
 sdks/python/apache_beam/runners/dataflow_runner_test.py | 5 ++---
 2 files changed, 6 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/56512ab4/sdks/python/apache_beam/runners/dataflow_runner.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/runners/dataflow_runner.py b/sdks/python/apache_beam/runners/dataflow_runner.py
index bd25dbf..31d3386 100644
--- a/sdks/python/apache_beam/runners/dataflow_runner.py
+++ b/sdks/python/apache_beam/runners/dataflow_runner.py
@@ -78,7 +78,7 @@ class DataflowRunner(PipelineRunner):
     return 's%s' % self._unique_step_id
 
   @staticmethod
-  def poll_for_job_completion(runner, job_id):
+  def poll_for_job_completion(runner, result):
     """Polls for the specified job to finish running (successfully or not)."""
     last_message_time = None
     last_message_id = None
@@ -101,6 +101,7 @@ class DataflowRunner(PipelineRunner):
       else:
         return 0
 
+    job_id = result.job_id()
     while True:
       response = runner.dataflow_client.get_job(job_id)
       # If get() is called very soon after Create() the response may not contain
@@ -151,6 +152,7 @@ class DataflowRunner(PipelineRunner):
         if not page_token:
           break
 
+    result._job = response
     runner.last_error_msg = last_error_msg
 
   def run(self, pipeline):
@@ -694,7 +696,7 @@ class DataflowPipelineResult(PipelineResult):
 
       thread = threading.Thread(
           target=DataflowRunner.poll_for_job_completion,
-          args=(self._runner, self.job_id()))
+          args=(self._runner, self))
 
       # Mark the thread as a daemon thread so a keyboard interrupt on the main
       # thread will terminate everything. This is also the reason we will not

http://git-wip-us.apache.org/repos/asf/beam/blob/56512ab4/sdks/python/apache_beam/runners/dataflow_runner_test.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/runners/dataflow_runner_test.py b/sdks/python/apache_beam/runners/dataflow_runner_test.py
index a935c98..4983899 100644
--- a/sdks/python/apache_beam/runners/dataflow_runner_test.py
+++ b/sdks/python/apache_beam/runners/dataflow_runner_test.py
@@ -47,12 +47,11 @@ class DataflowRunnerTest(unittest.TestCase):
         self.dataflow_client.list_messages = mock.MagicMock(
             return_value=([], None))
 
-    with self.assertRaises(DataflowRuntimeException) as e:
+    with self.assertRaisesRegexp(
+        DataflowRuntimeException, 'Dataflow pipeline failed. State: FAILED'):
       failed_runner = MockDataflowRunner(values_enum.JOB_STATE_FAILED)
       failed_result = DataflowPipelineResult(failed_runner.job, failed_runner)
       failed_result.wait_until_finish()
-    self.assertTrue(
-        'Dataflow pipeline failed. State: FAILED' in e.exception.message)
 
     succeeded_runner = MockDataflowRunner(values_enum.JOB_STATE_DONE)
     succeeded_result = DataflowPipelineResult(


[12/50] [abbrv] beam git commit: Code cleanup now that all runners support windowed side inputs.

Posted by da...@apache.org.
Code cleanup now that all runners support windowed side inputs.


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

Branch: refs/heads/master
Commit: 6cb2f37efadfb52138b125fcaf51e703c2c5fd5a
Parents: deb2aea
Author: Robert Bradshaw <ro...@gmail.com>
Authored: Sat Jan 21 21:13:36 2017 -0800
Committer: Robert Bradshaw <ro...@google.com>
Committed: Mon Jan 23 14:37:45 2017 -0800

----------------------------------------------------------------------
 sdks/python/apache_beam/runners/common.py | 34 ++++++++++++++------------
 1 file changed, 19 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/6cb2f37e/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 0f63cbc..9c8fdfc 100644
--- a/sdks/python/apache_beam/runners/common.py
+++ b/sdks/python/apache_beam/runners/common.py
@@ -24,7 +24,6 @@ import sys
 from apache_beam.internal import util
 from apache_beam.pvalue import SideOutputValue
 from apache_beam.transforms import core
-from apache_beam.transforms import sideinputs
 from apache_beam.transforms import window
 from apache_beam.transforms.window import TimestampedValue
 from apache_beam.transforms.window import WindowFn
@@ -71,6 +70,21 @@ class DoFnRunner(Receiver):
                # Preferred alternative to context
                # TODO(robertwb): Remove once all runners are updated.
                state=None):
+    """Initializes a DoFnRunner.
+
+    Args:
+      fn: user DoFn to invoke
+      args: positional side input arguments (static and placeholder), if any
+      kwargs: keyword side input arguments (static and placeholder), if any
+      side_inputs: list of sideinput.SideInputMaps for deferred side inputs
+      windowing: windowing properties of the output PCollection(s)
+      context: a DoFnContext to use (deprecated)
+      tagged_receivers: a dict of tag name to Receiver objects
+      logger: a logging module (deprecated)
+      step_name: the name of this step
+      logging_context: a LoggingContext object
+      state: handle for accessing DoFn state
+    """
     self.step_name = step_name
     self.window_fn = windowing.windowfn
     self.tagged_receivers = tagged_receivers
@@ -97,14 +111,10 @@ class DoFnRunner(Receiver):
     if isinstance(fn, core.NewDoFn):
       self.is_new_dofn = True
 
-      # SideInputs
-      self.side_inputs = [side_input
-                          if isinstance(side_input, sideinputs.SideInputMap)
-                          else {global_window: side_input}
-                          for side_input in side_inputs]
+      # Stash values for use in new_dofn_process.
+      self.side_inputs = side_inputs
       self.has_windowed_side_inputs = not all(
-          isinstance(si, dict) or si.is_globally_windowed()
-          for si in self.side_inputs)
+          si.is_globally_windowed() for si in self.side_inputs)
 
       self.args = args if args else []
       self.kwargs = kwargs if kwargs else {}
@@ -117,14 +127,8 @@ class DoFnRunner(Receiver):
         self.dofn = fn
         self.dofn_process = fn.process
       else:
-        # TODO(robertwb): Remove when all runners pass side input maps.
-        side_inputs = [side_input
-                       if isinstance(side_input, sideinputs.SideInputMap)
-                       else {global_window: side_input}
-                       for side_input in side_inputs]
         if side_inputs and all(
-            isinstance(side_input, dict) or side_input.is_globally_windowed()
-            for side_input in side_inputs):
+            side_input.is_globally_windowed() for side_input in side_inputs):
           args, kwargs = util.insert_values_in_args(
               args, kwargs, [side_input[global_window]
                              for side_input in side_inputs])


[15/50] [abbrv] beam git commit: Fix case where side inputs may be an iterable rather than a list.

Posted by da...@apache.org.
Fix case where side inputs may be an iterable rather than a list.


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

Branch: refs/heads/master
Commit: 9052366f0474b19c35b2838e6790e3333750e09e
Parents: d0dc1f3
Author: Robert Bradshaw <ro...@gmail.com>
Authored: Tue Jan 24 11:57:20 2017 -0800
Committer: Robert Bradshaw <ro...@gmail.com>
Committed: Tue Jan 24 11:57:20 2017 -0800

----------------------------------------------------------------------
 sdks/python/apache_beam/runners/common.py | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/9052366f/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 9c8fdfc..3741582 100644
--- a/sdks/python/apache_beam/runners/common.py
+++ b/sdks/python/apache_beam/runners/common.py
@@ -91,6 +91,9 @@ class DoFnRunner(Receiver):
 
     global_window = window.GlobalWindow()
 
+    # Need to support multiple iterations.
+    side_inputs = list(side_inputs)
+
     if logging_context:
       self.logging_context = logging_context
     else:


[32/50] [abbrv] beam git commit: Updating dataflow client protos to add new metrics.

Posted by da...@apache.org.
Updating dataflow client protos to add new metrics.

In order to use counter structured names, and add new metrics (e.g.
Distributions), we need to update the dataflow client protocol buffers.


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

Branch: refs/heads/master
Commit: 901a14c47bd21fadd563de1017f7c9d2b38cf4f1
Parents: 3d6f20d
Author: Pablo <pa...@google.com>
Authored: Thu Jan 26 18:02:04 2017 -0800
Committer: Pablo <pa...@google.com>
Committed: Fri Jan 27 11:46:26 2017 -0800

----------------------------------------------------------------------
 .../clients/dataflow/dataflow_v1b3_client.py    | 578 ++++++++----
 .../clients/dataflow/dataflow_v1b3_messages.py  | 931 +++++++++++++------
 2 files changed, 1075 insertions(+), 434 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/901a14c4/sdks/python/apache_beam/internal/clients/dataflow/dataflow_v1b3_client.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/internal/clients/dataflow/dataflow_v1b3_client.py b/sdks/python/apache_beam/internal/clients/dataflow/dataflow_v1b3_client.py
index 840b887..6ae2b73 100644
--- a/sdks/python/apache_beam/internal/clients/dataflow/dataflow_v1b3_client.py
+++ b/sdks/python/apache_beam/internal/clients/dataflow/dataflow_v1b3_client.py
@@ -55,6 +55,10 @@ class DataflowV1b3(base_api.BaseApiClient):
     self.projects_jobs_messages = self.ProjectsJobsMessagesService(self)
     self.projects_jobs_workItems = self.ProjectsJobsWorkItemsService(self)
     self.projects_jobs = self.ProjectsJobsService(self)
+    self.projects_locations_jobs_messages = self.ProjectsLocationsJobsMessagesService(self)
+    self.projects_locations_jobs_workItems = self.ProjectsLocationsJobsWorkItemsService(self)
+    self.projects_locations_jobs = self.ProjectsLocationsJobsService(self)
+    self.projects_locations = self.ProjectsLocationsService(self)
     self.projects_templates = self.ProjectsTemplatesService(self)
     self.projects = self.ProjectsService(self)
 
@@ -65,33 +69,6 @@ class DataflowV1b3(base_api.BaseApiClient):
 
     def __init__(self, client):
       super(DataflowV1b3.ProjectsJobsDebugService, self).__init__(client)
-      self._method_configs = {
-          'GetConfig': base_api.ApiMethodInfo(
-              http_method=u'POST',
-              method_id=u'dataflow.projects.jobs.debug.getConfig',
-              ordered_params=[u'projectId', u'jobId'],
-              path_params=[u'jobId', u'projectId'],
-              query_params=[],
-              relative_path=u'v1b3/projects/{projectId}/jobs/{jobId}/debug/getConfig',
-              request_field=u'getDebugConfigRequest',
-              request_type_name=u'DataflowProjectsJobsDebugGetConfigRequest',
-              response_type_name=u'GetDebugConfigResponse',
-              supports_download=False,
-          ),
-          'SendCapture': base_api.ApiMethodInfo(
-              http_method=u'POST',
-              method_id=u'dataflow.projects.jobs.debug.sendCapture',
-              ordered_params=[u'projectId', u'jobId'],
-              path_params=[u'jobId', u'projectId'],
-              query_params=[],
-              relative_path=u'v1b3/projects/{projectId}/jobs/{jobId}/debug/sendCapture',
-              request_field=u'sendDebugCaptureRequest',
-              request_type_name=u'DataflowProjectsJobsDebugSendCaptureRequest',
-              response_type_name=u'SendDebugCaptureResponse',
-              supports_download=False,
-          ),
-          }
-
       self._upload_configs = {
           }
 
@@ -108,6 +85,19 @@ class DataflowV1b3(base_api.BaseApiClient):
       return self._RunMethod(
           config, request, global_params=global_params)
 
+    GetConfig.method_config = lambda: base_api.ApiMethodInfo(
+        http_method=u'POST',
+        method_id=u'dataflow.projects.jobs.debug.getConfig',
+        ordered_params=[u'projectId', u'jobId'],
+        path_params=[u'jobId', u'projectId'],
+        query_params=[],
+        relative_path=u'v1b3/projects/{projectId}/jobs/{jobId}/debug/getConfig',
+        request_field=u'getDebugConfigRequest',
+        request_type_name=u'DataflowProjectsJobsDebugGetConfigRequest',
+        response_type_name=u'GetDebugConfigResponse',
+        supports_download=False,
+    )
+
     def SendCapture(self, request, global_params=None):
       """Send encoded debug capture data for component.
 
@@ -121,6 +111,19 @@ class DataflowV1b3(base_api.BaseApiClient):
       return self._RunMethod(
           config, request, global_params=global_params)
 
+    SendCapture.method_config = lambda: base_api.ApiMethodInfo(
+        http_method=u'POST',
+        method_id=u'dataflow.projects.jobs.debug.sendCapture',
+        ordered_params=[u'projectId', u'jobId'],
+        path_params=[u'jobId', u'projectId'],
+        query_params=[],
+        relative_path=u'v1b3/projects/{projectId}/jobs/{jobId}/debug/sendCapture',
+        request_field=u'sendDebugCaptureRequest',
+        request_type_name=u'DataflowProjectsJobsDebugSendCaptureRequest',
+        response_type_name=u'SendDebugCaptureResponse',
+        supports_download=False,
+    )
+
   class ProjectsJobsMessagesService(base_api.BaseApiService):
     """Service class for the projects_jobs_messages resource."""
 
@@ -128,21 +131,6 @@ class DataflowV1b3(base_api.BaseApiClient):
 
     def __init__(self, client):
       super(DataflowV1b3.ProjectsJobsMessagesService, self).__init__(client)
-      self._method_configs = {
-          'List': base_api.ApiMethodInfo(
-              http_method=u'GET',
-              method_id=u'dataflow.projects.jobs.messages.list',
-              ordered_params=[u'projectId', u'jobId'],
-              path_params=[u'jobId', u'projectId'],
-              query_params=[u'endTime', u'minimumImportance', u'pageSize', u'pageToken', u'startTime'],
-              relative_path=u'v1b3/projects/{projectId}/jobs/{jobId}/messages',
-              request_field='',
-              request_type_name=u'DataflowProjectsJobsMessagesListRequest',
-              response_type_name=u'ListJobMessagesResponse',
-              supports_download=False,
-          ),
-          }
-
       self._upload_configs = {
           }
 
@@ -159,6 +147,19 @@ class DataflowV1b3(base_api.BaseApiClient):
       return self._RunMethod(
           config, request, global_params=global_params)
 
+    List.method_config = lambda: base_api.ApiMethodInfo(
+        http_method=u'GET',
+        method_id=u'dataflow.projects.jobs.messages.list',
+        ordered_params=[u'projectId', u'jobId'],
+        path_params=[u'jobId', u'projectId'],
+        query_params=[u'endTime', u'location', u'minimumImportance', u'pageSize', u'pageToken', u'startTime'],
+        relative_path=u'v1b3/projects/{projectId}/jobs/{jobId}/messages',
+        request_field='',
+        request_type_name=u'DataflowProjectsJobsMessagesListRequest',
+        response_type_name=u'ListJobMessagesResponse',
+        supports_download=False,
+    )
+
   class ProjectsJobsWorkItemsService(base_api.BaseApiService):
     """Service class for the projects_jobs_workItems resource."""
 
@@ -166,33 +167,6 @@ class DataflowV1b3(base_api.BaseApiClient):
 
     def __init__(self, client):
       super(DataflowV1b3.ProjectsJobsWorkItemsService, self).__init__(client)
-      self._method_configs = {
-          'Lease': base_api.ApiMethodInfo(
-              http_method=u'POST',
-              method_id=u'dataflow.projects.jobs.workItems.lease',
-              ordered_params=[u'projectId', u'jobId'],
-              path_params=[u'jobId', u'projectId'],
-              query_params=[],
-              relative_path=u'v1b3/projects/{projectId}/jobs/{jobId}/workItems:lease',
-              request_field=u'leaseWorkItemRequest',
-              request_type_name=u'DataflowProjectsJobsWorkItemsLeaseRequest',
-              response_type_name=u'LeaseWorkItemResponse',
-              supports_download=False,
-          ),
-          'ReportStatus': base_api.ApiMethodInfo(
-              http_method=u'POST',
-              method_id=u'dataflow.projects.jobs.workItems.reportStatus',
-              ordered_params=[u'projectId', u'jobId'],
-              path_params=[u'jobId', u'projectId'],
-              query_params=[],
-              relative_path=u'v1b3/projects/{projectId}/jobs/{jobId}/workItems:reportStatus',
-              request_field=u'reportWorkItemStatusRequest',
-              request_type_name=u'DataflowProjectsJobsWorkItemsReportStatusRequest',
-              response_type_name=u'ReportWorkItemStatusResponse',
-              supports_download=False,
-          ),
-          }
-
       self._upload_configs = {
           }
 
@@ -209,6 +183,19 @@ class DataflowV1b3(base_api.BaseApiClient):
       return self._RunMethod(
           config, request, global_params=global_params)
 
+    Lease.method_config = lambda: base_api.ApiMethodInfo(
+        http_method=u'POST',
+        method_id=u'dataflow.projects.jobs.workItems.lease',
+        ordered_params=[u'projectId', u'jobId'],
+        path_params=[u'jobId', u'projectId'],
+        query_params=[],
+        relative_path=u'v1b3/projects/{projectId}/jobs/{jobId}/workItems:lease',
+        request_field=u'leaseWorkItemRequest',
+        request_type_name=u'DataflowProjectsJobsWorkItemsLeaseRequest',
+        response_type_name=u'LeaseWorkItemResponse',
+        supports_download=False,
+    )
+
     def ReportStatus(self, request, global_params=None):
       """Reports the status of dataflow WorkItems leased by a worker.
 
@@ -222,6 +209,19 @@ class DataflowV1b3(base_api.BaseApiClient):
       return self._RunMethod(
           config, request, global_params=global_params)
 
+    ReportStatus.method_config = lambda: base_api.ApiMethodInfo(
+        http_method=u'POST',
+        method_id=u'dataflow.projects.jobs.workItems.reportStatus',
+        ordered_params=[u'projectId', u'jobId'],
+        path_params=[u'jobId', u'projectId'],
+        query_params=[],
+        relative_path=u'v1b3/projects/{projectId}/jobs/{jobId}/workItems:reportStatus',
+        request_field=u'reportWorkItemStatusRequest',
+        request_type_name=u'DataflowProjectsJobsWorkItemsReportStatusRequest',
+        response_type_name=u'ReportWorkItemStatusResponse',
+        supports_download=False,
+    )
+
   class ProjectsJobsService(base_api.BaseApiService):
     """Service class for the projects_jobs resource."""
 
@@ -229,74 +229,11 @@ class DataflowV1b3(base_api.BaseApiClient):
 
     def __init__(self, client):
       super(DataflowV1b3.ProjectsJobsService, self).__init__(client)
-      self._method_configs = {
-          'Create': base_api.ApiMethodInfo(
-              http_method=u'POST',
-              method_id=u'dataflow.projects.jobs.create',
-              ordered_params=[u'projectId'],
-              path_params=[u'projectId'],
-              query_params=[u'replaceJobId', u'view'],
-              relative_path=u'v1b3/projects/{projectId}/jobs',
-              request_field=u'job',
-              request_type_name=u'DataflowProjectsJobsCreateRequest',
-              response_type_name=u'Job',
-              supports_download=False,
-          ),
-          'Get': base_api.ApiMethodInfo(
-              http_method=u'GET',
-              method_id=u'dataflow.projects.jobs.get',
-              ordered_params=[u'projectId', u'jobId'],
-              path_params=[u'jobId', u'projectId'],
-              query_params=[u'view'],
-              relative_path=u'v1b3/projects/{projectId}/jobs/{jobId}',
-              request_field='',
-              request_type_name=u'DataflowProjectsJobsGetRequest',
-              response_type_name=u'Job',
-              supports_download=False,
-          ),
-          'GetMetrics': base_api.ApiMethodInfo(
-              http_method=u'GET',
-              method_id=u'dataflow.projects.jobs.getMetrics',
-              ordered_params=[u'projectId', u'jobId'],
-              path_params=[u'jobId', u'projectId'],
-              query_params=[u'startTime'],
-              relative_path=u'v1b3/projects/{projectId}/jobs/{jobId}/metrics',
-              request_field='',
-              request_type_name=u'DataflowProjectsJobsGetMetricsRequest',
-              response_type_name=u'JobMetrics',
-              supports_download=False,
-          ),
-          'List': base_api.ApiMethodInfo(
-              http_method=u'GET',
-              method_id=u'dataflow.projects.jobs.list',
-              ordered_params=[u'projectId'],
-              path_params=[u'projectId'],
-              query_params=[u'filter', u'pageSize', u'pageToken', u'view'],
-              relative_path=u'v1b3/projects/{projectId}/jobs',
-              request_field='',
-              request_type_name=u'DataflowProjectsJobsListRequest',
-              response_type_name=u'ListJobsResponse',
-              supports_download=False,
-          ),
-          'Update': base_api.ApiMethodInfo(
-              http_method=u'PUT',
-              method_id=u'dataflow.projects.jobs.update',
-              ordered_params=[u'projectId', u'jobId'],
-              path_params=[u'jobId', u'projectId'],
-              query_params=[],
-              relative_path=u'v1b3/projects/{projectId}/jobs/{jobId}',
-              request_field=u'job',
-              request_type_name=u'DataflowProjectsJobsUpdateRequest',
-              response_type_name=u'Job',
-              supports_download=False,
-          ),
-          }
-
       self._upload_configs = {
           }
 
     def Create(self, request, global_params=None):
-      """Creates a dataflow job.
+      """Creates a Cloud Dataflow job.
 
       Args:
         request: (DataflowProjectsJobsCreateRequest) input message
@@ -308,8 +245,21 @@ class DataflowV1b3(base_api.BaseApiClient):
       return self._RunMethod(
           config, request, global_params=global_params)
 
+    Create.method_config = lambda: base_api.ApiMethodInfo(
+        http_method=u'POST',
+        method_id=u'dataflow.projects.jobs.create',
+        ordered_params=[u'projectId'],
+        path_params=[u'projectId'],
+        query_params=[u'location', u'replaceJobId', u'view'],
+        relative_path=u'v1b3/projects/{projectId}/jobs',
+        request_field=u'job',
+        request_type_name=u'DataflowProjectsJobsCreateRequest',
+        response_type_name=u'Job',
+        supports_download=False,
+    )
+
     def Get(self, request, global_params=None):
-      """Gets the state of the specified dataflow job.
+      """Gets the state of the specified Cloud Dataflow job.
 
       Args:
         request: (DataflowProjectsJobsGetRequest) input message
@@ -321,6 +271,19 @@ class DataflowV1b3(base_api.BaseApiClient):
       return self._RunMethod(
           config, request, global_params=global_params)
 
+    Get.method_config = lambda: base_api.ApiMethodInfo(
+        http_method=u'GET',
+        method_id=u'dataflow.projects.jobs.get',
+        ordered_params=[u'projectId', u'jobId'],
+        path_params=[u'jobId', u'projectId'],
+        query_params=[u'location', u'view'],
+        relative_path=u'v1b3/projects/{projectId}/jobs/{jobId}',
+        request_field='',
+        request_type_name=u'DataflowProjectsJobsGetRequest',
+        response_type_name=u'Job',
+        supports_download=False,
+    )
+
     def GetMetrics(self, request, global_params=None):
       """Request the job status.
 
@@ -334,6 +297,19 @@ class DataflowV1b3(base_api.BaseApiClient):
       return self._RunMethod(
           config, request, global_params=global_params)
 
+    GetMetrics.method_config = lambda: base_api.ApiMethodInfo(
+        http_method=u'GET',
+        method_id=u'dataflow.projects.jobs.getMetrics',
+        ordered_params=[u'projectId', u'jobId'],
+        path_params=[u'jobId', u'projectId'],
+        query_params=[u'location', u'startTime'],
+        relative_path=u'v1b3/projects/{projectId}/jobs/{jobId}/metrics',
+        request_field='',
+        request_type_name=u'DataflowProjectsJobsGetMetricsRequest',
+        response_type_name=u'JobMetrics',
+        supports_download=False,
+    )
+
     def List(self, request, global_params=None):
       """List the jobs of a project.
 
@@ -347,8 +323,21 @@ class DataflowV1b3(base_api.BaseApiClient):
       return self._RunMethod(
           config, request, global_params=global_params)
 
+    List.method_config = lambda: base_api.ApiMethodInfo(
+        http_method=u'GET',
+        method_id=u'dataflow.projects.jobs.list',
+        ordered_params=[u'projectId'],
+        path_params=[u'projectId'],
+        query_params=[u'filter', u'location', u'pageSize', u'pageToken', u'view'],
+        relative_path=u'v1b3/projects/{projectId}/jobs',
+        request_field='',
+        request_type_name=u'DataflowProjectsJobsListRequest',
+        response_type_name=u'ListJobsResponse',
+        supports_download=False,
+    )
+
     def Update(self, request, global_params=None):
-      """Updates the state of an existing dataflow job.
+      """Updates the state of an existing Cloud Dataflow job.
 
       Args:
         request: (DataflowProjectsJobsUpdateRequest) input message
@@ -360,6 +349,267 @@ class DataflowV1b3(base_api.BaseApiClient):
       return self._RunMethod(
           config, request, global_params=global_params)
 
+    Update.method_config = lambda: base_api.ApiMethodInfo(
+        http_method=u'PUT',
+        method_id=u'dataflow.projects.jobs.update',
+        ordered_params=[u'projectId', u'jobId'],
+        path_params=[u'jobId', u'projectId'],
+        query_params=[u'location'],
+        relative_path=u'v1b3/projects/{projectId}/jobs/{jobId}',
+        request_field=u'job',
+        request_type_name=u'DataflowProjectsJobsUpdateRequest',
+        response_type_name=u'Job',
+        supports_download=False,
+    )
+
+  class ProjectsLocationsJobsMessagesService(base_api.BaseApiService):
+    """Service class for the projects_locations_jobs_messages resource."""
+
+    _NAME = u'projects_locations_jobs_messages'
+
+    def __init__(self, client):
+      super(DataflowV1b3.ProjectsLocationsJobsMessagesService, self).__init__(client)
+      self._upload_configs = {
+          }
+
+    def List(self, request, global_params=None):
+      """Request the job status.
+
+      Args:
+        request: (DataflowProjectsLocationsJobsMessagesListRequest) input message
+        global_params: (StandardQueryParameters, default: None) global arguments
+      Returns:
+        (ListJobMessagesResponse) The response message.
+      """
+      config = self.GetMethodConfig('List')
+      return self._RunMethod(
+          config, request, global_params=global_params)
+
+    List.method_config = lambda: base_api.ApiMethodInfo(
+        http_method=u'GET',
+        method_id=u'dataflow.projects.locations.jobs.messages.list',
+        ordered_params=[u'projectId', u'location', u'jobId'],
+        path_params=[u'jobId', u'location', u'projectId'],
+        query_params=[u'endTime', u'minimumImportance', u'pageSize', u'pageToken', u'startTime'],
+        relative_path=u'v1b3/projects/{projectId}/locations/{location}/jobs/{jobId}/messages',
+        request_field='',
+        request_type_name=u'DataflowProjectsLocationsJobsMessagesListRequest',
+        response_type_name=u'ListJobMessagesResponse',
+        supports_download=False,
+    )
+
+  class ProjectsLocationsJobsWorkItemsService(base_api.BaseApiService):
+    """Service class for the projects_locations_jobs_workItems resource."""
+
+    _NAME = u'projects_locations_jobs_workItems'
+
+    def __init__(self, client):
+      super(DataflowV1b3.ProjectsLocationsJobsWorkItemsService, self).__init__(client)
+      self._upload_configs = {
+          }
+
+    def Lease(self, request, global_params=None):
+      """Leases a dataflow WorkItem to run.
+
+      Args:
+        request: (DataflowProjectsLocationsJobsWorkItemsLeaseRequest) input message
+        global_params: (StandardQueryParameters, default: None) global arguments
+      Returns:
+        (LeaseWorkItemResponse) The response message.
+      """
+      config = self.GetMethodConfig('Lease')
+      return self._RunMethod(
+          config, request, global_params=global_params)
+
+    Lease.method_config = lambda: base_api.ApiMethodInfo(
+        http_method=u'POST',
+        method_id=u'dataflow.projects.locations.jobs.workItems.lease',
+        ordered_params=[u'projectId', u'location', u'jobId'],
+        path_params=[u'jobId', u'location', u'projectId'],
+        query_params=[],
+        relative_path=u'v1b3/projects/{projectId}/locations/{location}/jobs/{jobId}/workItems:lease',
+        request_field=u'leaseWorkItemRequest',
+        request_type_name=u'DataflowProjectsLocationsJobsWorkItemsLeaseRequest',
+        response_type_name=u'LeaseWorkItemResponse',
+        supports_download=False,
+    )
+
+    def ReportStatus(self, request, global_params=None):
+      """Reports the status of dataflow WorkItems leased by a worker.
+
+      Args:
+        request: (DataflowProjectsLocationsJobsWorkItemsReportStatusRequest) input message
+        global_params: (StandardQueryParameters, default: None) global arguments
+      Returns:
+        (ReportWorkItemStatusResponse) The response message.
+      """
+      config = self.GetMethodConfig('ReportStatus')
+      return self._RunMethod(
+          config, request, global_params=global_params)
+
+    ReportStatus.method_config = lambda: base_api.ApiMethodInfo(
+        http_method=u'POST',
+        method_id=u'dataflow.projects.locations.jobs.workItems.reportStatus',
+        ordered_params=[u'projectId', u'location', u'jobId'],
+        path_params=[u'jobId', u'location', u'projectId'],
+        query_params=[],
+        relative_path=u'v1b3/projects/{projectId}/locations/{location}/jobs/{jobId}/workItems:reportStatus',
+        request_field=u'reportWorkItemStatusRequest',
+        request_type_name=u'DataflowProjectsLocationsJobsWorkItemsReportStatusRequest',
+        response_type_name=u'ReportWorkItemStatusResponse',
+        supports_download=False,
+    )
+
+  class ProjectsLocationsJobsService(base_api.BaseApiService):
+    """Service class for the projects_locations_jobs resource."""
+
+    _NAME = u'projects_locations_jobs'
+
+    def __init__(self, client):
+      super(DataflowV1b3.ProjectsLocationsJobsService, self).__init__(client)
+      self._upload_configs = {
+          }
+
+    def Create(self, request, global_params=None):
+      """Creates a Cloud Dataflow job.
+
+      Args:
+        request: (DataflowProjectsLocationsJobsCreateRequest) input message
+        global_params: (StandardQueryParameters, default: None) global arguments
+      Returns:
+        (Job) The response message.
+      """
+      config = self.GetMethodConfig('Create')
+      return self._RunMethod(
+          config, request, global_params=global_params)
+
+    Create.method_config = lambda: base_api.ApiMethodInfo(
+        http_method=u'POST',
+        method_id=u'dataflow.projects.locations.jobs.create',
+        ordered_params=[u'projectId', u'location'],
+        path_params=[u'location', u'projectId'],
+        query_params=[u'replaceJobId', u'view'],
+        relative_path=u'v1b3/projects/{projectId}/locations/{location}/jobs',
+        request_field=u'job',
+        request_type_name=u'DataflowProjectsLocationsJobsCreateRequest',
+        response_type_name=u'Job',
+        supports_download=False,
+    )
+
+    def Get(self, request, global_params=None):
+      """Gets the state of the specified Cloud Dataflow job.
+
+      Args:
+        request: (DataflowProjectsLocationsJobsGetRequest) input message
+        global_params: (StandardQueryParameters, default: None) global arguments
+      Returns:
+        (Job) The response message.
+      """
+      config = self.GetMethodConfig('Get')
+      return self._RunMethod(
+          config, request, global_params=global_params)
+
+    Get.method_config = lambda: base_api.ApiMethodInfo(
+        http_method=u'GET',
+        method_id=u'dataflow.projects.locations.jobs.get',
+        ordered_params=[u'projectId', u'location', u'jobId'],
+        path_params=[u'jobId', u'location', u'projectId'],
+        query_params=[u'view'],
+        relative_path=u'v1b3/projects/{projectId}/locations/{location}/jobs/{jobId}',
+        request_field='',
+        request_type_name=u'DataflowProjectsLocationsJobsGetRequest',
+        response_type_name=u'Job',
+        supports_download=False,
+    )
+
+    def GetMetrics(self, request, global_params=None):
+      """Request the job status.
+
+      Args:
+        request: (DataflowProjectsLocationsJobsGetMetricsRequest) input message
+        global_params: (StandardQueryParameters, default: None) global arguments
+      Returns:
+        (JobMetrics) The response message.
+      """
+      config = self.GetMethodConfig('GetMetrics')
+      return self._RunMethod(
+          config, request, global_params=global_params)
+
+    GetMetrics.method_config = lambda: base_api.ApiMethodInfo(
+        http_method=u'GET',
+        method_id=u'dataflow.projects.locations.jobs.getMetrics',
+        ordered_params=[u'projectId', u'location', u'jobId'],
+        path_params=[u'jobId', u'location', u'projectId'],
+        query_params=[u'startTime'],
+        relative_path=u'v1b3/projects/{projectId}/locations/{location}/jobs/{jobId}/metrics',
+        request_field='',
+        request_type_name=u'DataflowProjectsLocationsJobsGetMetricsRequest',
+        response_type_name=u'JobMetrics',
+        supports_download=False,
+    )
+
+    def List(self, request, global_params=None):
+      """List the jobs of a project.
+
+      Args:
+        request: (DataflowProjectsLocationsJobsListRequest) input message
+        global_params: (StandardQueryParameters, default: None) global arguments
+      Returns:
+        (ListJobsResponse) The response message.
+      """
+      config = self.GetMethodConfig('List')
+      return self._RunMethod(
+          config, request, global_params=global_params)
+
+    List.method_config = lambda: base_api.ApiMethodInfo(
+        http_method=u'GET',
+        method_id=u'dataflow.projects.locations.jobs.list',
+        ordered_params=[u'projectId', u'location'],
+        path_params=[u'location', u'projectId'],
+        query_params=[u'filter', u'pageSize', u'pageToken', u'view'],
+        relative_path=u'v1b3/projects/{projectId}/locations/{location}/jobs',
+        request_field='',
+        request_type_name=u'DataflowProjectsLocationsJobsListRequest',
+        response_type_name=u'ListJobsResponse',
+        supports_download=False,
+    )
+
+    def Update(self, request, global_params=None):
+      """Updates the state of an existing Cloud Dataflow job.
+
+      Args:
+        request: (DataflowProjectsLocationsJobsUpdateRequest) input message
+        global_params: (StandardQueryParameters, default: None) global arguments
+      Returns:
+        (Job) The response message.
+      """
+      config = self.GetMethodConfig('Update')
+      return self._RunMethod(
+          config, request, global_params=global_params)
+
+    Update.method_config = lambda: base_api.ApiMethodInfo(
+        http_method=u'PUT',
+        method_id=u'dataflow.projects.locations.jobs.update',
+        ordered_params=[u'projectId', u'location', u'jobId'],
+        path_params=[u'jobId', u'location', u'projectId'],
+        query_params=[],
+        relative_path=u'v1b3/projects/{projectId}/locations/{location}/jobs/{jobId}',
+        request_field=u'job',
+        request_type_name=u'DataflowProjectsLocationsJobsUpdateRequest',
+        response_type_name=u'Job',
+        supports_download=False,
+    )
+
+  class ProjectsLocationsService(base_api.BaseApiService):
+    """Service class for the projects_locations resource."""
+
+    _NAME = u'projects_locations'
+
+    def __init__(self, client):
+      super(DataflowV1b3.ProjectsLocationsService, self).__init__(client)
+      self._upload_configs = {
+          }
+
   class ProjectsTemplatesService(base_api.BaseApiService):
     """Service class for the projects_templates resource."""
 
@@ -367,26 +617,11 @@ class DataflowV1b3(base_api.BaseApiClient):
 
     def __init__(self, client):
       super(DataflowV1b3.ProjectsTemplatesService, self).__init__(client)
-      self._method_configs = {
-          'Create': base_api.ApiMethodInfo(
-              http_method=u'POST',
-              method_id=u'dataflow.projects.templates.create',
-              ordered_params=[u'projectId'],
-              path_params=[u'projectId'],
-              query_params=[],
-              relative_path=u'v1b3/projects/{projectId}/templates',
-              request_field=u'createJobFromTemplateRequest',
-              request_type_name=u'DataflowProjectsTemplatesCreateRequest',
-              response_type_name=u'Job',
-              supports_download=False,
-          ),
-          }
-
       self._upload_configs = {
           }
 
     def Create(self, request, global_params=None):
-      """Creates a dataflow job from a template.
+      """Creates a Cloud Dataflow job from a template.
 
       Args:
         request: (DataflowProjectsTemplatesCreateRequest) input message
@@ -398,6 +633,19 @@ class DataflowV1b3(base_api.BaseApiClient):
       return self._RunMethod(
           config, request, global_params=global_params)
 
+    Create.method_config = lambda: base_api.ApiMethodInfo(
+        http_method=u'POST',
+        method_id=u'dataflow.projects.templates.create',
+        ordered_params=[u'projectId'],
+        path_params=[u'projectId'],
+        query_params=[],
+        relative_path=u'v1b3/projects/{projectId}/templates',
+        request_field=u'createJobFromTemplateRequest',
+        request_type_name=u'DataflowProjectsTemplatesCreateRequest',
+        response_type_name=u'Job',
+        supports_download=False,
+    )
+
   class ProjectsService(base_api.BaseApiService):
     """Service class for the projects resource."""
 
@@ -405,21 +653,6 @@ class DataflowV1b3(base_api.BaseApiClient):
 
     def __init__(self, client):
       super(DataflowV1b3.ProjectsService, self).__init__(client)
-      self._method_configs = {
-          'WorkerMessages': base_api.ApiMethodInfo(
-              http_method=u'POST',
-              method_id=u'dataflow.projects.workerMessages',
-              ordered_params=[u'projectId'],
-              path_params=[u'projectId'],
-              query_params=[],
-              relative_path=u'v1b3/projects/{projectId}/WorkerMessages',
-              request_field=u'sendWorkerMessagesRequest',
-              request_type_name=u'DataflowProjectsWorkerMessagesRequest',
-              response_type_name=u'SendWorkerMessagesResponse',
-              supports_download=False,
-          ),
-          }
-
       self._upload_configs = {
           }
 
@@ -435,3 +668,16 @@ class DataflowV1b3(base_api.BaseApiClient):
       config = self.GetMethodConfig('WorkerMessages')
       return self._RunMethod(
           config, request, global_params=global_params)
+
+    WorkerMessages.method_config = lambda: base_api.ApiMethodInfo(
+        http_method=u'POST',
+        method_id=u'dataflow.projects.workerMessages',
+        ordered_params=[u'projectId'],
+        path_params=[u'projectId'],
+        query_params=[],
+        relative_path=u'v1b3/projects/{projectId}/WorkerMessages',
+        request_field=u'sendWorkerMessagesRequest',
+        request_type_name=u'DataflowProjectsWorkerMessagesRequest',
+        response_type_name=u'SendWorkerMessagesResponse',
+        supports_download=False,
+    )


[02/50] [abbrv] beam git commit: Closes #1809

Posted by da...@apache.org.
Closes #1809


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

Branch: refs/heads/master
Commit: 946135f6a955d9e27e7553c4cefef354ecd2535d
Parents: c03e6f3 56512ab
Author: Robert Bradshaw <ro...@gmail.com>
Authored: Sat Jan 21 00:30:35 2017 -0800
Committer: Robert Bradshaw <ro...@gmail.com>
Committed: Sat Jan 21 00:30:35 2017 -0800

----------------------------------------------------------------------
 sdks/python/apache_beam/runners/dataflow_runner.py      | 6 ++++--
 sdks/python/apache_beam/runners/dataflow_runner_test.py | 5 ++---
 2 files changed, 6 insertions(+), 5 deletions(-)
----------------------------------------------------------------------



[16/50] [abbrv] beam git commit: Closes #1832

Posted by da...@apache.org.
Closes #1832


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

Branch: refs/heads/master
Commit: f9831236c4a94af35a0a40a649323b578a3d92e4
Parents: d0dc1f3 9052366
Author: Robert Bradshaw <ro...@gmail.com>
Authored: Tue Jan 24 13:48:12 2017 -0800
Committer: Robert Bradshaw <ro...@gmail.com>
Committed: Tue Jan 24 13:48:12 2017 -0800

----------------------------------------------------------------------
 sdks/python/apache_beam/runners/common.py | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------



[14/50] [abbrv] beam git commit: Closes #1831

Posted by da...@apache.org.
Closes #1831


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

Branch: refs/heads/master
Commit: d0dc1f375982bab747eda8ea26f4a41b15a1ec01
Parents: af49908 96fcc7d
Author: Robert Bradshaw <ro...@gmail.com>
Authored: Tue Jan 24 10:06:03 2017 -0800
Committer: Robert Bradshaw <ro...@gmail.com>
Committed: Tue Jan 24 10:06:03 2017 -0800

----------------------------------------------------------------------
 sdks/python/apache_beam/dataflow_test.py        | 418 +++++++++++++++++++
 .../apache_beam/transforms/ptransform_test.py   |  67 ---
 .../apache_beam/transforms/sideinputs_test.py   | 208 +--------
 3 files changed, 419 insertions(+), 274 deletions(-)
----------------------------------------------------------------------



[45/50] [abbrv] beam git commit: This closes #1863

Posted by da...@apache.org.
This closes #1863


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

Branch: refs/heads/master
Commit: 1390699c37596ebe34a773627660b6c496375a8e
Parents: 475707f e02ddac
Author: Davor Bonaci <da...@google.com>
Authored: Mon Jan 30 12:45:03 2017 -0800
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jan 30 12:45:03 2017 -0800

----------------------------------------------------------------------
 sdks/python/apache_beam/io/bigquery_test.py | 26 ++++++++++++++++--------
 1 file changed, 18 insertions(+), 8 deletions(-)
----------------------------------------------------------------------



[41/50] [abbrv] beam git commit: This closes #1870

Posted by da...@apache.org.
This closes #1870


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

Branch: refs/heads/master
Commit: f29527f68b8de92caf18b183e3a7e97eb190f67e
Parents: 27cf68e 38575a1
Author: Davor Bonaci <da...@google.com>
Authored: Mon Jan 30 12:38:53 2017 -0800
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jan 30 12:38:53 2017 -0800

----------------------------------------------------------------------
 .../GroupAlsoByWindowViaWindowSetNewDoFn.java   | 156 +++++++++++++++++++
 .../wrappers/streaming/DoFnOperator.java        |  69 ++++----
 .../wrappers/streaming/WindowDoFnOperator.java  | 143 +++++++++--------
 sdks/python/apache_beam/version.py              |   3 +-
 4 files changed, 265 insertions(+), 106 deletions(-)
----------------------------------------------------------------------



[31/50] [abbrv] beam git commit: Updating dataflow client protos to add new metrics.

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam/blob/901a14c4/sdks/python/apache_beam/internal/clients/dataflow/dataflow_v1b3_messages.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/internal/clients/dataflow/dataflow_v1b3_messages.py b/sdks/python/apache_beam/internal/clients/dataflow/dataflow_v1b3_messages.py
index 178a542..a42154e 100644
--- a/sdks/python/apache_beam/internal/clients/dataflow/dataflow_v1b3_messages.py
+++ b/sdks/python/apache_beam/internal/clients/dataflow/dataflow_v1b3_messages.py
@@ -24,6 +24,7 @@ and continuous computation.
 
 from apitools.base.protorpclite import messages as _messages
 from apitools.base.py import encoding
+from apitools.base.py import extra_types
 
 
 package = 'dataflow'
@@ -193,6 +194,7 @@ class CounterMetadata(_messages.Message):
       AND: Aggregated value represents the logical 'and' of all contributed
         values.
       SET: Aggregated value is a set of unique contributed values.
+      DISTRIBUTION: Aggregated value captures statistics about a distribution.
     """
     INVALID = 0
     SUM = 1
@@ -202,6 +204,7 @@ class CounterMetadata(_messages.Message):
     OR = 5
     AND = 6
     SET = 7
+    DISTRIBUTION = 8
 
   class StandardUnitsValueValuesEnum(_messages.Enum):
     """System defined Units, see above enum.
@@ -308,6 +311,7 @@ class CounterUpdate(_messages.Message):
       aggregate value accumulated since the worker started working on this
       WorkItem. By default this is false, indicating that this counter is
       reported as a delta.
+    distribution: Distribution data
     floatingPoint: Floating point value for Sum, Max, Min.
     floatingPointList: List of floating point numbers, for Set.
     floatingPointMean: Floating point mean aggregation value for Mean.
@@ -326,34 +330,38 @@ class CounterUpdate(_messages.Message):
 
   boolean = _messages.BooleanField(1)
   cumulative = _messages.BooleanField(2)
-  floatingPoint = _messages.FloatField(3)
-  floatingPointList = _messages.MessageField('FloatingPointList', 4)
-  floatingPointMean = _messages.MessageField('FloatingPointMean', 5)
-  integer = _messages.MessageField('SplitInt64', 6)
-  integerList = _messages.MessageField('IntegerList', 7)
-  integerMean = _messages.MessageField('IntegerMean', 8)
-  internal = _messages.MessageField('extra_types.JsonValue', 9)
-  nameAndKind = _messages.MessageField('NameAndKind', 10)
-  shortId = _messages.IntegerField(11)
-  stringList = _messages.MessageField('StringList', 12)
-  structuredNameAndMetadata = _messages.MessageField('CounterStructuredNameAndMetadata', 13)
+  distribution = _messages.MessageField('DistributionUpdate', 3)
+  floatingPoint = _messages.FloatField(4)
+  floatingPointList = _messages.MessageField('FloatingPointList', 5)
+  floatingPointMean = _messages.MessageField('FloatingPointMean', 6)
+  integer = _messages.MessageField('SplitInt64', 7)
+  integerList = _messages.MessageField('IntegerList', 8)
+  integerMean = _messages.MessageField('IntegerMean', 9)
+  internal = _messages.MessageField('extra_types.JsonValue', 10)
+  nameAndKind = _messages.MessageField('NameAndKind', 11)
+  shortId = _messages.IntegerField(12)
+  stringList = _messages.MessageField('StringList', 13)
+  structuredNameAndMetadata = _messages.MessageField('CounterStructuredNameAndMetadata', 14)
 
 
 class CreateJobFromTemplateRequest(_messages.Message):
-  """Request to create a Dataflow job.
+  """A request to create a Cloud Dataflow job from a template.
 
   Messages:
-    ParametersValue: Dynamic parameterization of the job's runtime
-      environment.
+    ParametersValue: The runtime parameters to pass to the job.
 
   Fields:
-    gcsPath: A path to the serialized JSON representation of the job.
-    parameters: Dynamic parameterization of the job's runtime environment.
+    environment: The runtime environment for the job.
+    gcsPath: Required. A Cloud Storage path to the template from which to
+      create the job. Must be a valid Cloud Storage URL, beginning with
+      `gs://`.
+    jobName: Required. The job name to use for the created job.
+    parameters: The runtime parameters to pass to the job.
   """
 
   @encoding.MapUnrecognizedFields('additionalProperties')
   class ParametersValue(_messages.Message):
-    """Dynamic parameterization of the job's runtime environment.
+    """The runtime parameters to pass to the job.
 
     Messages:
       AdditionalProperty: An additional property for a ParametersValue object.
@@ -375,8 +383,10 @@ class CreateJobFromTemplateRequest(_messages.Message):
 
     additionalProperties = _messages.MessageField('AdditionalProperty', 1, repeated=True)
 
-  gcsPath = _messages.StringField(1)
-  parameters = _messages.MessageField('ParametersValue', 2)
+  environment = _messages.MessageField('RuntimeEnvironment', 1)
+  gcsPath = _messages.StringField(2)
+  jobName = _messages.StringField(3)
+  parameters = _messages.MessageField('ParametersValue', 4)
 
 
 class CustomSourceLocation(_messages.Message):
@@ -410,17 +420,18 @@ class DataflowProjectsJobsCreateRequest(_messages.Message):
   """A DataflowProjectsJobsCreateRequest object.
 
   Enums:
-    ViewValueValuesEnum: Level of information requested in response.
+    ViewValueValuesEnum: The level of information requested in response.
 
   Fields:
     job: A Job resource to be passed as the request body.
-    projectId: The project which owns the job.
-    replaceJobId: DEPRECATED. This field is now on the Job message.
-    view: Level of information requested in response.
+    location: The location that contains this job.
+    projectId: The ID of the Cloud Platform project that the job belongs to.
+    replaceJobId: Deprecated. This field is now in the Job message.
+    view: The level of information requested in response.
   """
 
   class ViewValueValuesEnum(_messages.Enum):
-    """Level of information requested in response.
+    """The level of information requested in response.
 
     Values:
       JOB_VIEW_UNKNOWN: <no description>
@@ -432,9 +443,10 @@ class DataflowProjectsJobsCreateRequest(_messages.Message):
     JOB_VIEW_ALL = 2
 
   job = _messages.MessageField('Job', 1)
-  projectId = _messages.StringField(2, required=True)
-  replaceJobId = _messages.StringField(3)
-  view = _messages.EnumField('ViewValueValuesEnum', 4)
+  location = _messages.StringField(2)
+  projectId = _messages.StringField(3, required=True)
+  replaceJobId = _messages.StringField(4)
+  view = _messages.EnumField('ViewValueValuesEnum', 5)
 
 
 class DataflowProjectsJobsDebugGetConfigRequest(_messages.Message):
@@ -472,30 +484,33 @@ class DataflowProjectsJobsGetMetricsRequest(_messages.Message):
 
   Fields:
     jobId: The job to get messages for.
+    location: The location which contains the job specified by job_id.
     projectId: A project id.
     startTime: Return only metric data that has changed since this time.
       Default is to return all information about all metrics for the job.
   """
 
   jobId = _messages.StringField(1, required=True)
-  projectId = _messages.StringField(2, required=True)
-  startTime = _messages.StringField(3)
+  location = _messages.StringField(2)
+  projectId = _messages.StringField(3, required=True)
+  startTime = _messages.StringField(4)
 
 
 class DataflowProjectsJobsGetRequest(_messages.Message):
   """A DataflowProjectsJobsGetRequest object.
 
   Enums:
-    ViewValueValuesEnum: Level of information requested in response.
+    ViewValueValuesEnum: The level of information requested in response.
 
   Fields:
-    jobId: Identifies a single job.
-    projectId: The project which owns the job.
-    view: Level of information requested in response.
+    jobId: The job ID.
+    location: The location that contains this job.
+    projectId: The ID of the Cloud Platform project that the job belongs to.
+    view: The level of information requested in response.
   """
 
   class ViewValueValuesEnum(_messages.Enum):
-    """Level of information requested in response.
+    """The level of information requested in response.
 
     Values:
       JOB_VIEW_UNKNOWN: <no description>
@@ -507,8 +522,9 @@ class DataflowProjectsJobsGetRequest(_messages.Message):
     JOB_VIEW_ALL = 2
 
   jobId = _messages.StringField(1, required=True)
-  projectId = _messages.StringField(2, required=True)
-  view = _messages.EnumField('ViewValueValuesEnum', 3)
+  location = _messages.StringField(2)
+  projectId = _messages.StringField(3, required=True)
+  view = _messages.EnumField('ViewValueValuesEnum', 4)
 
 
 class DataflowProjectsJobsListRequest(_messages.Message):
@@ -517,17 +533,19 @@ class DataflowProjectsJobsListRequest(_messages.Message):
   Enums:
     FilterValueValuesEnum: The kind of filter to use.
     ViewValueValuesEnum: Level of information requested in response. Default
-      is SUMMARY.
+      is `JOB_VIEW_SUMMARY`.
 
   Fields:
     filter: The kind of filter to use.
+    location: The location that contains this job.
     pageSize: If there are many jobs, limit response to at most this many. The
       actual number of jobs returned will be the lesser of max_responses and
       an unspecified server-defined limit.
     pageToken: Set this to the 'next_page_token' field of a previous response
       to request additional results in a long list.
     projectId: The project which owns the jobs.
-    view: Level of information requested in response. Default is SUMMARY.
+    view: Level of information requested in response. Default is
+      `JOB_VIEW_SUMMARY`.
   """
 
   class FilterValueValuesEnum(_messages.Enum):
@@ -545,7 +563,8 @@ class DataflowProjectsJobsListRequest(_messages.Message):
     ACTIVE = 3
 
   class ViewValueValuesEnum(_messages.Enum):
-    """Level of information requested in response. Default is SUMMARY.
+    """Level of information requested in response. Default is
+    `JOB_VIEW_SUMMARY`.
 
     Values:
       JOB_VIEW_UNKNOWN: <no description>
@@ -557,10 +576,11 @@ class DataflowProjectsJobsListRequest(_messages.Message):
     JOB_VIEW_ALL = 2
 
   filter = _messages.EnumField('FilterValueValuesEnum', 1)
-  pageSize = _messages.IntegerField(2, variant=_messages.Variant.INT32)
-  pageToken = _messages.StringField(3)
-  projectId = _messages.StringField(4, required=True)
-  view = _messages.EnumField('ViewValueValuesEnum', 5)
+  location = _messages.StringField(2)
+  pageSize = _messages.IntegerField(3, variant=_messages.Variant.INT32)
+  pageToken = _messages.StringField(4)
+  projectId = _messages.StringField(5, required=True)
+  view = _messages.EnumField('ViewValueValuesEnum', 6)
 
 
 class DataflowProjectsJobsMessagesListRequest(_messages.Message):
@@ -574,6 +594,7 @@ class DataflowProjectsJobsMessagesListRequest(_messages.Message):
     endTime: Return only messages with timestamps < end_time. The default is
       now (i.e. return up to the latest messages available).
     jobId: The job to get messages about.
+    location: The location which contains the job specified by job_id.
     minimumImportance: Filter to only get messages with importance >= level
     pageSize: If specified, determines the maximum number of messages to
       return.  If unspecified, the service may choose an appropriate default,
@@ -607,11 +628,12 @@ class DataflowProjectsJobsMessagesListRequest(_messages.Message):
 
   endTime = _messages.StringField(1)
   jobId = _messages.StringField(2, required=True)
-  minimumImportance = _messages.EnumField('MinimumImportanceValueValuesEnum', 3)
-  pageSize = _messages.IntegerField(4, variant=_messages.Variant.INT32)
-  pageToken = _messages.StringField(5)
-  projectId = _messages.StringField(6, required=True)
-  startTime = _messages.StringField(7)
+  location = _messages.StringField(3)
+  minimumImportance = _messages.EnumField('MinimumImportanceValueValuesEnum', 4)
+  pageSize = _messages.IntegerField(5, variant=_messages.Variant.INT32)
+  pageToken = _messages.StringField(6)
+  projectId = _messages.StringField(7, required=True)
+  startTime = _messages.StringField(8)
 
 
 class DataflowProjectsJobsUpdateRequest(_messages.Message):
@@ -619,13 +641,15 @@ class DataflowProjectsJobsUpdateRequest(_messages.Message):
 
   Fields:
     job: A Job resource to be passed as the request body.
-    jobId: Identifies a single job.
-    projectId: The project which owns the job.
+    jobId: The job ID.
+    location: The location that contains this job.
+    projectId: The ID of the Cloud Platform project that the job belongs to.
   """
 
   job = _messages.MessageField('Job', 1)
   jobId = _messages.StringField(2, required=True)
-  projectId = _messages.StringField(3, required=True)
+  location = _messages.StringField(3)
+  projectId = _messages.StringField(4, required=True)
 
 
 class DataflowProjectsJobsWorkItemsLeaseRequest(_messages.Message):
@@ -658,13 +682,254 @@ class DataflowProjectsJobsWorkItemsReportStatusRequest(_messages.Message):
   reportWorkItemStatusRequest = _messages.MessageField('ReportWorkItemStatusRequest', 3)
 
 
+class DataflowProjectsLocationsJobsCreateRequest(_messages.Message):
+  """A DataflowProjectsLocationsJobsCreateRequest object.
+
+  Enums:
+    ViewValueValuesEnum: The level of information requested in response.
+
+  Fields:
+    job: A Job resource to be passed as the request body.
+    location: The location that contains this job.
+    projectId: The ID of the Cloud Platform project that the job belongs to.
+    replaceJobId: Deprecated. This field is now in the Job message.
+    view: The level of information requested in response.
+  """
+
+  class ViewValueValuesEnum(_messages.Enum):
+    """The level of information requested in response.
+
+    Values:
+      JOB_VIEW_UNKNOWN: <no description>
+      JOB_VIEW_SUMMARY: <no description>
+      JOB_VIEW_ALL: <no description>
+    """
+    JOB_VIEW_UNKNOWN = 0
+    JOB_VIEW_SUMMARY = 1
+    JOB_VIEW_ALL = 2
+
+  job = _messages.MessageField('Job', 1)
+  location = _messages.StringField(2, required=True)
+  projectId = _messages.StringField(3, required=True)
+  replaceJobId = _messages.StringField(4)
+  view = _messages.EnumField('ViewValueValuesEnum', 5)
+
+
+class DataflowProjectsLocationsJobsGetMetricsRequest(_messages.Message):
+  """A DataflowProjectsLocationsJobsGetMetricsRequest object.
+
+  Fields:
+    jobId: The job to get messages for.
+    location: The location which contains the job specified by job_id.
+    projectId: A project id.
+    startTime: Return only metric data that has changed since this time.
+      Default is to return all information about all metrics for the job.
+  """
+
+  jobId = _messages.StringField(1, required=True)
+  location = _messages.StringField(2, required=True)
+  projectId = _messages.StringField(3, required=True)
+  startTime = _messages.StringField(4)
+
+
+class DataflowProjectsLocationsJobsGetRequest(_messages.Message):
+  """A DataflowProjectsLocationsJobsGetRequest object.
+
+  Enums:
+    ViewValueValuesEnum: The level of information requested in response.
+
+  Fields:
+    jobId: The job ID.
+    location: The location that contains this job.
+    projectId: The ID of the Cloud Platform project that the job belongs to.
+    view: The level of information requested in response.
+  """
+
+  class ViewValueValuesEnum(_messages.Enum):
+    """The level of information requested in response.
+
+    Values:
+      JOB_VIEW_UNKNOWN: <no description>
+      JOB_VIEW_SUMMARY: <no description>
+      JOB_VIEW_ALL: <no description>
+    """
+    JOB_VIEW_UNKNOWN = 0
+    JOB_VIEW_SUMMARY = 1
+    JOB_VIEW_ALL = 2
+
+  jobId = _messages.StringField(1, required=True)
+  location = _messages.StringField(2, required=True)
+  projectId = _messages.StringField(3, required=True)
+  view = _messages.EnumField('ViewValueValuesEnum', 4)
+
+
+class DataflowProjectsLocationsJobsListRequest(_messages.Message):
+  """A DataflowProjectsLocationsJobsListRequest object.
+
+  Enums:
+    FilterValueValuesEnum: The kind of filter to use.
+    ViewValueValuesEnum: Level of information requested in response. Default
+      is `JOB_VIEW_SUMMARY`.
+
+  Fields:
+    filter: The kind of filter to use.
+    location: The location that contains this job.
+    pageSize: If there are many jobs, limit response to at most this many. The
+      actual number of jobs returned will be the lesser of max_responses and
+      an unspecified server-defined limit.
+    pageToken: Set this to the 'next_page_token' field of a previous response
+      to request additional results in a long list.
+    projectId: The project which owns the jobs.
+    view: Level of information requested in response. Default is
+      `JOB_VIEW_SUMMARY`.
+  """
+
+  class FilterValueValuesEnum(_messages.Enum):
+    """The kind of filter to use.
+
+    Values:
+      UNKNOWN: <no description>
+      ALL: <no description>
+      TERMINATED: <no description>
+      ACTIVE: <no description>
+    """
+    UNKNOWN = 0
+    ALL = 1
+    TERMINATED = 2
+    ACTIVE = 3
+
+  class ViewValueValuesEnum(_messages.Enum):
+    """Level of information requested in response. Default is
+    `JOB_VIEW_SUMMARY`.
+
+    Values:
+      JOB_VIEW_UNKNOWN: <no description>
+      JOB_VIEW_SUMMARY: <no description>
+      JOB_VIEW_ALL: <no description>
+    """
+    JOB_VIEW_UNKNOWN = 0
+    JOB_VIEW_SUMMARY = 1
+    JOB_VIEW_ALL = 2
+
+  filter = _messages.EnumField('FilterValueValuesEnum', 1)
+  location = _messages.StringField(2, required=True)
+  pageSize = _messages.IntegerField(3, variant=_messages.Variant.INT32)
+  pageToken = _messages.StringField(4)
+  projectId = _messages.StringField(5, required=True)
+  view = _messages.EnumField('ViewValueValuesEnum', 6)
+
+
+class DataflowProjectsLocationsJobsMessagesListRequest(_messages.Message):
+  """A DataflowProjectsLocationsJobsMessagesListRequest object.
+
+  Enums:
+    MinimumImportanceValueValuesEnum: Filter to only get messages with
+      importance >= level
+
+  Fields:
+    endTime: Return only messages with timestamps < end_time. The default is
+      now (i.e. return up to the latest messages available).
+    jobId: The job to get messages about.
+    location: The location which contains the job specified by job_id.
+    minimumImportance: Filter to only get messages with importance >= level
+    pageSize: If specified, determines the maximum number of messages to
+      return.  If unspecified, the service may choose an appropriate default,
+      or may return an arbitrarily large number of results.
+    pageToken: If supplied, this should be the value of next_page_token
+      returned by an earlier call. This will cause the next page of results to
+      be returned.
+    projectId: A project id.
+    startTime: If specified, return only messages with timestamps >=
+      start_time. The default is the job creation time (i.e. beginning of
+      messages).
+  """
+
+  class MinimumImportanceValueValuesEnum(_messages.Enum):
+    """Filter to only get messages with importance >= level
+
+    Values:
+      JOB_MESSAGE_IMPORTANCE_UNKNOWN: <no description>
+      JOB_MESSAGE_DEBUG: <no description>
+      JOB_MESSAGE_DETAILED: <no description>
+      JOB_MESSAGE_BASIC: <no description>
+      JOB_MESSAGE_WARNING: <no description>
+      JOB_MESSAGE_ERROR: <no description>
+    """
+    JOB_MESSAGE_IMPORTANCE_UNKNOWN = 0
+    JOB_MESSAGE_DEBUG = 1
+    JOB_MESSAGE_DETAILED = 2
+    JOB_MESSAGE_BASIC = 3
+    JOB_MESSAGE_WARNING = 4
+    JOB_MESSAGE_ERROR = 5
+
+  endTime = _messages.StringField(1)
+  jobId = _messages.StringField(2, required=True)
+  location = _messages.StringField(3, required=True)
+  minimumImportance = _messages.EnumField('MinimumImportanceValueValuesEnum', 4)
+  pageSize = _messages.IntegerField(5, variant=_messages.Variant.INT32)
+  pageToken = _messages.StringField(6)
+  projectId = _messages.StringField(7, required=True)
+  startTime = _messages.StringField(8)
+
+
+class DataflowProjectsLocationsJobsUpdateRequest(_messages.Message):
+  """A DataflowProjectsLocationsJobsUpdateRequest object.
+
+  Fields:
+    job: A Job resource to be passed as the request body.
+    jobId: The job ID.
+    location: The location that contains this job.
+    projectId: The ID of the Cloud Platform project that the job belongs to.
+  """
+
+  job = _messages.MessageField('Job', 1)
+  jobId = _messages.StringField(2, required=True)
+  location = _messages.StringField(3, required=True)
+  projectId = _messages.StringField(4, required=True)
+
+
+class DataflowProjectsLocationsJobsWorkItemsLeaseRequest(_messages.Message):
+  """A DataflowProjectsLocationsJobsWorkItemsLeaseRequest object.
+
+  Fields:
+    jobId: Identifies the workflow job this worker belongs to.
+    leaseWorkItemRequest: A LeaseWorkItemRequest resource to be passed as the
+      request body.
+    location: The location which contains the WorkItem's job.
+    projectId: Identifies the project this worker belongs to.
+  """
+
+  jobId = _messages.StringField(1, required=True)
+  leaseWorkItemRequest = _messages.MessageField('LeaseWorkItemRequest', 2)
+  location = _messages.StringField(3, required=True)
+  projectId = _messages.StringField(4, required=True)
+
+
+class DataflowProjectsLocationsJobsWorkItemsReportStatusRequest(_messages.Message):
+  """A DataflowProjectsLocationsJobsWorkItemsReportStatusRequest object.
+
+  Fields:
+    jobId: The job which the WorkItem is part of.
+    location: The location which contains the WorkItem's job.
+    projectId: The project which owns the WorkItem's job.
+    reportWorkItemStatusRequest: A ReportWorkItemStatusRequest resource to be
+      passed as the request body.
+  """
+
+  jobId = _messages.StringField(1, required=True)
+  location = _messages.StringField(2, required=True)
+  projectId = _messages.StringField(3, required=True)
+  reportWorkItemStatusRequest = _messages.MessageField('ReportWorkItemStatusRequest', 4)
+
+
 class DataflowProjectsTemplatesCreateRequest(_messages.Message):
   """A DataflowProjectsTemplatesCreateRequest object.
 
   Fields:
     createJobFromTemplateRequest: A CreateJobFromTemplateRequest resource to
       be passed as the request body.
-    projectId: The project which owns the job.
+    projectId: Required. The ID of the Cloud Platform project that the job
+      belongs to.
   """
 
   createJobFromTemplateRequest = _messages.MessageField('CreateJobFromTemplateRequest', 1)
@@ -732,12 +997,12 @@ class Disk(_messages.Message):
       disk type is a resource name typically ending in "pd-standard".  If SSD
       persistent disks are available, the resource name typically ends with
       "pd-ssd".  The actual valid values are defined the Google Compute Engine
-      API, not by the Dataflow API; consult the Google Compute Engine
+      API, not by the Cloud Dataflow API; consult the Google Compute Engine
       documentation for more information about determining the set of
       available disk types for a particular project and zone.  Google Compute
       Engine Disk types are local to a particular project in a particular
       zone, and so the resource name will typically look something like this:
-      compute.googleapis.com/projects/<project-id>/zones/<zone>/diskTypes/pd-
+      compute.googleapis.com/projects/project-id/zones/zone/diskTypes/pd-
       standard
     mountPoint: Directory in a VM where disk is mounted.
     sizeGb: Size of disk in GB.  If zero or unspecified, the service will
@@ -749,6 +1014,26 @@ class Disk(_messages.Message):
   sizeGb = _messages.IntegerField(3, variant=_messages.Variant.INT32)
 
 
+class DistributionUpdate(_messages.Message):
+  """A metric value representing a distribution.
+
+  Fields:
+    count: The count of the number of elements present in the distribution.
+    max: The maximum value present in the distribution.
+    min: The minimum value present in the distribution.
+    sum: Use an int64 since we'd prefer the added precision. If overflow is a
+      common problem we can detect it and use an additional int64 or a double.
+    sumOfSquares: Use a double since the sum of squares is likely to overflow
+      int64.
+  """
+
+  count = _messages.MessageField('SplitInt64', 1)
+  max = _messages.MessageField('SplitInt64', 2)
+  min = _messages.MessageField('SplitInt64', 3)
+  sum = _messages.MessageField('SplitInt64', 4)
+  sumOfSquares = _messages.FloatField(5)
+
+
 class DynamicSourceSplit(_messages.Message):
   """When a task splits using WorkItemStatus.dynamic_source_split, this
   message describes the two parts of the split relative to the description of
@@ -770,10 +1055,10 @@ class Environment(_messages.Message):
 
   Messages:
     InternalExperimentsValue: Experimental settings.
-    SdkPipelineOptionsValue: The Dataflow SDK pipeline options specified by
-      the user. These options are passed through the service and are used to
-      recreate the SDK pipeline options on the worker in a language agnostic
-      and platform independent way.
+    SdkPipelineOptionsValue: The Cloud Dataflow SDK pipeline options specified
+      by the user. These options are passed through the service and are used
+      to recreate the SDK pipeline options on the worker in a language
+      agnostic and platform independent way.
     UserAgentValue: A description of the process that generated the request.
     VersionValue: A structure describing which components and their versions
       of the service are required in order to run the job.
@@ -788,8 +1073,8 @@ class Environment(_messages.Message):
       BigQuery:   bigquery.googleapis.com/{dataset}
     experiments: The list of experiments to enable.
     internalExperiments: Experimental settings.
-    sdkPipelineOptions: The Dataflow SDK pipeline options specified by the
-      user. These options are passed through the service and are used to
+    sdkPipelineOptions: The Cloud Dataflow SDK pipeline options specified by
+      the user. These options are passed through the service and are used to
       recreate the SDK pipeline options on the worker in a language agnostic
       and platform independent way.
     serviceAccountEmail: Identity to run virtual machines as. Defaults to the
@@ -806,7 +1091,7 @@ class Environment(_messages.Message):
     userAgent: A description of the process that generated the request.
     version: A structure describing which components and their versions of the
       service are required in order to run the job.
-    workerPools: Worker pools.  At least one "harness" worker pool must be
+    workerPools: The worker pools. At least one "harness" worker pool must be
       specified in order for the job to have workers.
   """
 
@@ -838,9 +1123,10 @@ class Environment(_messages.Message):
 
   @encoding.MapUnrecognizedFields('additionalProperties')
   class SdkPipelineOptionsValue(_messages.Message):
-    """The Dataflow SDK pipeline options specified by the user. These options
-    are passed through the service and are used to recreate the SDK pipeline
-    options on the worker in a language agnostic and platform independent way.
+    """The Cloud Dataflow SDK pipeline options specified by the user. These
+    options are passed through the service and are used to recreate the SDK
+    pipeline options on the worker in a language agnostic and platform
+    independent way.
 
     Messages:
       AdditionalProperty: An additional property for a SdkPipelineOptionsValue
@@ -924,6 +1210,16 @@ class Environment(_messages.Message):
   workerPools = _messages.MessageField('WorkerPool', 10, repeated=True)
 
 
+class FailedLocation(_messages.Message):
+  """Indicates which location failed to respond to a request for data.
+
+  Fields:
+    name: The name of the failed location.
+  """
+
+  name = _messages.StringField(1)
+
+
 class FlattenInstruction(_messages.Message):
   """An instruction that copies its inputs (zero or more) to its (single)
   output.
@@ -1071,22 +1367,22 @@ class IntegerMean(_messages.Message):
 
 
 class Job(_messages.Message):
-  """Defines a job to be run by the Dataflow service.
+  """Defines a job to be run by the Cloud Dataflow service.
 
   Enums:
     CurrentStateValueValuesEnum: The current state of the job.  Jobs are
-      created in the JOB_STATE_STOPPED state unless otherwise specified.  A
-      job in the JOB_STATE_RUNNING state may asynchronously enter a terminal
-      state.  Once a job has reached a terminal state, no further state
-      updates may be made.  This field may be mutated by the Dataflow service;
-      callers cannot mutate it.
-    RequestedStateValueValuesEnum: The job's requested state.  UpdateJob may
-      be used to switch between the JOB_STATE_STOPPED and JOB_STATE_RUNNING
-      states, by setting requested_state.  UpdateJob may also be used to
-      directly set a job's requested state to JOB_STATE_CANCELLED or
-      JOB_STATE_DONE, irrevocably terminating the job if it has not already
-      reached a terminal state.
-    TypeValueValuesEnum: The type of dataflow job.
+      created in the `JOB_STATE_STOPPED` state unless otherwise specified.  A
+      job in the `JOB_STATE_RUNNING` state may asynchronously enter a terminal
+      state. After a job has reached a terminal state, no further state
+      updates may be made.  This field may be mutated by the Cloud Dataflow
+      service; callers cannot mutate it.
+    RequestedStateValueValuesEnum: The job's requested state.  `UpdateJob` may
+      be used to switch between the `JOB_STATE_STOPPED` and
+      `JOB_STATE_RUNNING` states, by setting requested_state.  `UpdateJob` may
+      also be used to directly set a job's requested state to
+      `JOB_STATE_CANCELLED` or `JOB_STATE_DONE`, irrevocably terminating the
+      job if it has not already reached a terminal state.
+    TypeValueValuesEnum: The type of Cloud Dataflow job.
 
   Messages:
     LabelsValue: User-defined labels for this job.  The labels map can contain
@@ -1095,57 +1391,57 @@ class Job(_messages.Message):
       regexp:  \p{Ll}\p{Lo}{0,62} * Values must conform to regexp:
       [\p{Ll}\p{Lo}\p{N}_-]{0,63} * Both keys and values are additionally
       constrained to be <= 128 bytes in size.
-    TransformNameMappingValue: Map of transform name prefixes of the job to be
-      replaced to the corresponding name prefixes of the new job.
+    TransformNameMappingValue: The map of transform name prefixes of the job
+      to be replaced to the corresponding name prefixes of the new job.
 
   Fields:
-    clientRequestId: Client's unique identifier of the job, re-used by SDK
-      across retried attempts. If this field is set, the service will ensure
-      its uniqueness. That is, the request to create a job will fail if the
-      service has knowledge of a previously submitted job with the same
-      client's id and job name. The caller may, for example, use this field to
-      ensure idempotence of job creation across retried attempts to create a
-      job. By default, the field is empty and, in that case, the service
-      ignores it.
-    createTime: Timestamp when job was initially created. Immutable, set by
-      the Dataflow service.
+    clientRequestId: The client's unique identifier of the job, re-used across
+      retried attempts. If this field is set, the service will ensure its
+      uniqueness. The request to create a job will fail if the service has
+      knowledge of a previously submitted job with the same client's ID and
+      job name. The caller may use this field to ensure idempotence of job
+      creation across retried attempts to create a job. By default, the field
+      is empty and, in that case, the service ignores it.
+    createTime: The timestamp when the job was initially created. Immutable
+      and set by the Cloud Dataflow service.
     currentState: The current state of the job.  Jobs are created in the
-      JOB_STATE_STOPPED state unless otherwise specified.  A job in the
-      JOB_STATE_RUNNING state may asynchronously enter a terminal state.  Once
-      a job has reached a terminal state, no further state updates may be
-      made.  This field may be mutated by the Dataflow service; callers cannot
-      mutate it.
+      `JOB_STATE_STOPPED` state unless otherwise specified.  A job in the
+      `JOB_STATE_RUNNING` state may asynchronously enter a terminal state.
+      After a job has reached a terminal state, no further state updates may
+      be made.  This field may be mutated by the Cloud Dataflow service;
+      callers cannot mutate it.
     currentStateTime: The timestamp associated with the current state.
-    environment: Environment for the job.
-    executionInfo: Information about how the Dataflow service will actually
-      run the job.
-    id: The unique ID of this job.  This field is set by the Dataflow service
-      when the Job is created, and is immutable for the life of the Job.
+    environment: The environment for the job.
+    executionInfo: Information about how the Cloud Dataflow service will run
+      the job.
+    id: The unique ID of this job.  This field is set by the Cloud Dataflow
+      service when the Job is created, and is immutable for the life of the
+      job.
     labels: User-defined labels for this job.  The labels map can contain no
       more than 64 entries.  Entries of the labels map are UTF8 strings that
       comply with the following restrictions:  * Keys must conform to regexp:
       \p{Ll}\p{Lo}{0,62} * Values must conform to regexp:
       [\p{Ll}\p{Lo}\p{N}_-]{0,63} * Both keys and values are additionally
       constrained to be <= 128 bytes in size.
-    name: The user-specified Dataflow job name.  Only one Job with a given
-      name may exist in a project at any given time.  If a caller attempts to
-      create a Job with the same name as an already-existing Job, the attempt
-      will return the existing Job.  The name must match the regular
-      expression [a-z]([-a-z0-9]{0,38}[a-z0-9])?
-    projectId: The project which owns the job.
-    replaceJobId: If this job is an update of an existing job, this field will
-      be the ID of the job it replaced.  When sending a CreateJobRequest, you
-      can update a job by specifying it here. The job named here will be
-      stopped, and its intermediate state transferred to this job.
+    location: The location that contains this job.
+    name: The user-specified Cloud Dataflow job name.  Only one Job with a
+      given name may exist in a project at any given time. If a caller
+      attempts to create a Job with the same name as an already-existing Job,
+      the attempt returns the existing Job.  The name must match the regular
+      expression `[a-z]([-a-z0-9]{0,38}[a-z0-9])?`
+    projectId: The ID of the Cloud Platform project that the job belongs to.
+    replaceJobId: If this job is an update of an existing job, this field is
+      the job ID of the job it replaced.  When sending a `CreateJobRequest`,
+      you can update a job by specifying it here. The job named here is
+      stopped, and its intermediate state is transferred to this job.
     replacedByJobId: If another job is an update of this job (and thus, this
-      job is in JOB_STATE_UPDATED), this field will contain the ID of that
-      job.
-    requestedState: The job's requested state.  UpdateJob may be used to
-      switch between the JOB_STATE_STOPPED and JOB_STATE_RUNNING states, by
-      setting requested_state.  UpdateJob may also be used to directly set a
-      job's requested state to JOB_STATE_CANCELLED or JOB_STATE_DONE,
-      irrevocably terminating the job if it has not already reached a terminal
-      state.
+      job is in `JOB_STATE_UPDATED`), this field contains the ID of that job.
+    requestedState: The job's requested state.  `UpdateJob` may be used to
+      switch between the `JOB_STATE_STOPPED` and `JOB_STATE_RUNNING` states,
+      by setting requested_state.  `UpdateJob` may also be used to directly
+      set a job's requested state to `JOB_STATE_CANCELLED` or
+      `JOB_STATE_DONE`, irrevocably terminating the job if it has not already
+      reached a terminal state.
     steps: The top-level steps that constitute the entire job.
     tempFiles: A set of files the system should be aware of that are used for
       temporary storage. These temporary files will be removed on job
@@ -1153,53 +1449,55 @@ class Job(_messages.Message):
       The supported files are:  Google Cloud Storage:
       storage.googleapis.com/{bucket}/{object}
       bucket.storage.googleapis.com/{object}
-    transformNameMapping: Map of transform name prefixes of the job to be
+    transformNameMapping: The map of transform name prefixes of the job to be
       replaced to the corresponding name prefixes of the new job.
-    type: The type of dataflow job.
+    type: The type of Cloud Dataflow job.
   """
 
   class CurrentStateValueValuesEnum(_messages.Enum):
     """The current state of the job.  Jobs are created in the
-    JOB_STATE_STOPPED state unless otherwise specified.  A job in the
-    JOB_STATE_RUNNING state may asynchronously enter a terminal state.  Once a
-    job has reached a terminal state, no further state updates may be made.
-    This field may be mutated by the Dataflow service; callers cannot mutate
-    it.
+    `JOB_STATE_STOPPED` state unless otherwise specified.  A job in the
+    `JOB_STATE_RUNNING` state may asynchronously enter a terminal state. After
+    a job has reached a terminal state, no further state updates may be made.
+    This field may be mutated by the Cloud Dataflow service; callers cannot
+    mutate it.
 
     Values:
       JOB_STATE_UNKNOWN: The job's run state isn't specified.
-      JOB_STATE_STOPPED: JOB_STATE_STOPPED indicates that the job is paused,
-        or has not yet started to run.
-      JOB_STATE_RUNNING: JOB_STATE_RUNNING indicates that the job is currently
-        running.
-      JOB_STATE_DONE: JOB_STATE_DONE indicates that the job has successfully
+      JOB_STATE_STOPPED: `JOB_STATE_STOPPED` indicates that the job has not
+        yet started to run.
+      JOB_STATE_RUNNING: `JOB_STATE_RUNNING` indicates that the job is
+        currently running.
+      JOB_STATE_DONE: `JOB_STATE_DONE` indicates that the job has successfully
         completed. This is a terminal job state.  This state may be set by the
-        Dataflow service, as a transition from JOB_STATE_RUNNING. It may also
-        be set via a Dataflow UpdateJob call, if the job has not yet reached a
-        terminal state.
-      JOB_STATE_FAILED: JOB_STATE_FAILED indicates that the job has failed.
-        This is a terminal job state.  This state may only be set by the
-        Dataflow service, and only as a transition from JOB_STATE_RUNNING.
-      JOB_STATE_CANCELLED: JOB_STATE_CANCELLED indicates that the job has been
-        explicitly cancelled.  This is a terminal job state.  This state may
-        only be set via a Dataflow UpdateJob call, and only if the job has not
-        yet reached another terminal state.
-      JOB_STATE_UPDATED: JOB_STATE_UPDATED indicates that the job was
+        Cloud Dataflow service, as a transition from `JOB_STATE_RUNNING`. It
+        may also be set via a Cloud Dataflow `UpdateJob` call, if the job has
+        not yet reached a terminal state.
+      JOB_STATE_FAILED: `JOB_STATE_FAILED` indicates that the job has failed.
+        This is a terminal job state.  This state may only be set by the Cloud
+        Dataflow service, and only as a transition from `JOB_STATE_RUNNING`.
+      JOB_STATE_CANCELLED: `JOB_STATE_CANCELLED` indicates that the job has
+        been explicitly cancelled. This is a terminal job state. This state
+        may only be set via a Cloud Dataflow `UpdateJob` call, and only if the
+        job has not yet reached another terminal state.
+      JOB_STATE_UPDATED: `JOB_STATE_UPDATED` indicates that the job was
         successfully updated, meaning that this job was stopped and another
         job was started, inheriting state from this one. This is a terminal
-        job state. This state may only be set by the Dataflow service, and
-        only as a transition from JOB_STATE_RUNNING.
-      JOB_STATE_DRAINING: JOB_STATE_DRAINING indicates that the job is in the
-        process of draining. A draining job has stopped pulling from its input
-        sources and is processing any data that remains in-flight. This state
-        may be set via a Dataflow UpdateJob call, but only as a transition
-        from JOB_STATE_RUNNING. Jobs that are draining may only transition to
-        JOB_STATE_DRAINED, JOB_STATE_CANCELLED, or JOB_STATE_FAILED.
-      JOB_STATE_DRAINED: JOB_STATE_DRAINED indicates that the job has been
+        job state. This state may only be set by the Cloud Dataflow service,
+        and only as a transition from `JOB_STATE_RUNNING`.
+      JOB_STATE_DRAINING: `JOB_STATE_DRAINING` indicates that the job is in
+        the process of draining. A draining job has stopped pulling from its
+        input sources and is processing any data that remains in-flight. This
+        state may be set via a Cloud Dataflow `UpdateJob` call, but only as a
+        transition from `JOB_STATE_RUNNING`. Jobs that are draining may only
+        transition to `JOB_STATE_DRAINED`, `JOB_STATE_CANCELLED`, or
+        `JOB_STATE_FAILED`.
+      JOB_STATE_DRAINED: `JOB_STATE_DRAINED` indicates that the job has been
         drained. A drained job terminated by stopping pulling from its input
         sources and processing any data that remained in-flight when draining
         was requested. This state is a terminal state, may only be set by the
-        Dataflow service, and only as a transition from JOB_STATE_DRAINING.
+        Cloud Dataflow service, and only as a transition from
+        `JOB_STATE_DRAINING`.
     """
     JOB_STATE_UNKNOWN = 0
     JOB_STATE_STOPPED = 1
@@ -1212,46 +1510,48 @@ class Job(_messages.Message):
     JOB_STATE_DRAINED = 8
 
   class RequestedStateValueValuesEnum(_messages.Enum):
-    """The job's requested state.  UpdateJob may be used to switch between the
-    JOB_STATE_STOPPED and JOB_STATE_RUNNING states, by setting
-    requested_state.  UpdateJob may also be used to directly set a job's
-    requested state to JOB_STATE_CANCELLED or JOB_STATE_DONE, irrevocably
+    """The job's requested state.  `UpdateJob` may be used to switch between
+    the `JOB_STATE_STOPPED` and `JOB_STATE_RUNNING` states, by setting
+    requested_state.  `UpdateJob` may also be used to directly set a job's
+    requested state to `JOB_STATE_CANCELLED` or `JOB_STATE_DONE`, irrevocably
     terminating the job if it has not already reached a terminal state.
 
     Values:
       JOB_STATE_UNKNOWN: The job's run state isn't specified.
-      JOB_STATE_STOPPED: JOB_STATE_STOPPED indicates that the job is paused,
-        or has not yet started to run.
-      JOB_STATE_RUNNING: JOB_STATE_RUNNING indicates that the job is currently
-        running.
-      JOB_STATE_DONE: JOB_STATE_DONE indicates that the job has successfully
+      JOB_STATE_STOPPED: `JOB_STATE_STOPPED` indicates that the job has not
+        yet started to run.
+      JOB_STATE_RUNNING: `JOB_STATE_RUNNING` indicates that the job is
+        currently running.
+      JOB_STATE_DONE: `JOB_STATE_DONE` indicates that the job has successfully
         completed. This is a terminal job state.  This state may be set by the
-        Dataflow service, as a transition from JOB_STATE_RUNNING. It may also
-        be set via a Dataflow UpdateJob call, if the job has not yet reached a
-        terminal state.
-      JOB_STATE_FAILED: JOB_STATE_FAILED indicates that the job has failed.
-        This is a terminal job state.  This state may only be set by the
-        Dataflow service, and only as a transition from JOB_STATE_RUNNING.
-      JOB_STATE_CANCELLED: JOB_STATE_CANCELLED indicates that the job has been
-        explicitly cancelled.  This is a terminal job state.  This state may
-        only be set via a Dataflow UpdateJob call, and only if the job has not
-        yet reached another terminal state.
-      JOB_STATE_UPDATED: JOB_STATE_UPDATED indicates that the job was
+        Cloud Dataflow service, as a transition from `JOB_STATE_RUNNING`. It
+        may also be set via a Cloud Dataflow `UpdateJob` call, if the job has
+        not yet reached a terminal state.
+      JOB_STATE_FAILED: `JOB_STATE_FAILED` indicates that the job has failed.
+        This is a terminal job state.  This state may only be set by the Cloud
+        Dataflow service, and only as a transition from `JOB_STATE_RUNNING`.
+      JOB_STATE_CANCELLED: `JOB_STATE_CANCELLED` indicates that the job has
+        been explicitly cancelled. This is a terminal job state. This state
+        may only be set via a Cloud Dataflow `UpdateJob` call, and only if the
+        job has not yet reached another terminal state.
+      JOB_STATE_UPDATED: `JOB_STATE_UPDATED` indicates that the job was
         successfully updated, meaning that this job was stopped and another
         job was started, inheriting state from this one. This is a terminal
-        job state. This state may only be set by the Dataflow service, and
-        only as a transition from JOB_STATE_RUNNING.
-      JOB_STATE_DRAINING: JOB_STATE_DRAINING indicates that the job is in the
-        process of draining. A draining job has stopped pulling from its input
-        sources and is processing any data that remains in-flight. This state
-        may be set via a Dataflow UpdateJob call, but only as a transition
-        from JOB_STATE_RUNNING. Jobs that are draining may only transition to
-        JOB_STATE_DRAINED, JOB_STATE_CANCELLED, or JOB_STATE_FAILED.
-      JOB_STATE_DRAINED: JOB_STATE_DRAINED indicates that the job has been
+        job state. This state may only be set by the Cloud Dataflow service,
+        and only as a transition from `JOB_STATE_RUNNING`.
+      JOB_STATE_DRAINING: `JOB_STATE_DRAINING` indicates that the job is in
+        the process of draining. A draining job has stopped pulling from its
+        input sources and is processing any data that remains in-flight. This
+        state may be set via a Cloud Dataflow `UpdateJob` call, but only as a
+        transition from `JOB_STATE_RUNNING`. Jobs that are draining may only
+        transition to `JOB_STATE_DRAINED`, `JOB_STATE_CANCELLED`, or
+        `JOB_STATE_FAILED`.
+      JOB_STATE_DRAINED: `JOB_STATE_DRAINED` indicates that the job has been
         drained. A drained job terminated by stopping pulling from its input
         sources and processing any data that remained in-flight when draining
         was requested. This state is a terminal state, may only be set by the
-        Dataflow service, and only as a transition from JOB_STATE_DRAINING.
+        Cloud Dataflow service, and only as a transition from
+        `JOB_STATE_DRAINING`.
     """
     JOB_STATE_UNKNOWN = 0
     JOB_STATE_STOPPED = 1
@@ -1264,7 +1564,7 @@ class Job(_messages.Message):
     JOB_STATE_DRAINED = 8
 
   class TypeValueValuesEnum(_messages.Enum):
-    """The type of dataflow job.
+    """The type of Cloud Dataflow job.
 
     Values:
       JOB_TYPE_UNKNOWN: The type of the job is unspecified, or unknown.
@@ -1308,7 +1608,7 @@ class Job(_messages.Message):
 
   @encoding.MapUnrecognizedFields('additionalProperties')
   class TransformNameMappingValue(_messages.Message):
-    """Map of transform name prefixes of the job to be replaced to the
+    """The map of transform name prefixes of the job to be replaced to the
     corresponding name prefixes of the new job.
 
     Messages:
@@ -1341,20 +1641,21 @@ class Job(_messages.Message):
   executionInfo = _messages.MessageField('JobExecutionInfo', 6)
   id = _messages.StringField(7)
   labels = _messages.MessageField('LabelsValue', 8)
-  name = _messages.StringField(9)
-  projectId = _messages.StringField(10)
-  replaceJobId = _messages.StringField(11)
-  replacedByJobId = _messages.StringField(12)
-  requestedState = _messages.EnumField('RequestedStateValueValuesEnum', 13)
-  steps = _messages.MessageField('Step', 14, repeated=True)
-  tempFiles = _messages.StringField(15, repeated=True)
-  transformNameMapping = _messages.MessageField('TransformNameMappingValue', 16)
-  type = _messages.EnumField('TypeValueValuesEnum', 17)
+  location = _messages.StringField(9)
+  name = _messages.StringField(10)
+  projectId = _messages.StringField(11)
+  replaceJobId = _messages.StringField(12)
+  replacedByJobId = _messages.StringField(13)
+  requestedState = _messages.EnumField('RequestedStateValueValuesEnum', 14)
+  steps = _messages.MessageField('Step', 15, repeated=True)
+  tempFiles = _messages.StringField(16, repeated=True)
+  transformNameMapping = _messages.MessageField('TransformNameMappingValue', 17)
+  type = _messages.EnumField('TypeValueValuesEnum', 18)
 
 
 class JobExecutionInfo(_messages.Message):
-  """Additional information about how a Dataflow job will be executed which
-  isn\u2019t contained in the submitted job.
+  """Additional information about how a Cloud Dataflow job will be executed
+  that isn't contained in the submitted job.
 
   Messages:
     StagesValue: A mapping from each stage to the information about that
@@ -1525,6 +1826,7 @@ class LeaseWorkItemRequest(_messages.Message):
 
   Fields:
     currentWorkerTime: The current timestamp at the worker.
+    location: The location which contains the WorkItem's job.
     requestedLeaseDuration: The initial lease period.
     workItemTypes: Filter for WorkItem type.
     workerCapabilities: Worker capabilities. WorkItems might be limited to
@@ -1534,10 +1836,11 @@ class LeaseWorkItemRequest(_messages.Message):
   """
 
   currentWorkerTime = _messages.StringField(1)
-  requestedLeaseDuration = _messages.StringField(2)
-  workItemTypes = _messages.StringField(3, repeated=True)
-  workerCapabilities = _messages.StringField(4, repeated=True)
-  workerId = _messages.StringField(5)
+  location = _messages.StringField(2)
+  requestedLeaseDuration = _messages.StringField(3)
+  workItemTypes = _messages.StringField(4, repeated=True)
+  workerCapabilities = _messages.StringField(5, repeated=True)
+  workerId = _messages.StringField(6)
 
 
 class LeaseWorkItemResponse(_messages.Message):
@@ -1564,16 +1867,19 @@ class ListJobMessagesResponse(_messages.Message):
 
 
 class ListJobsResponse(_messages.Message):
-  """Response to a request to list Dataflow jobs.  This may be a partial
+  """Response to a request to list Cloud Dataflow jobs.  This may be a partial
   response, depending on the page size in the ListJobsRequest.
 
   Fields:
+    failedLocation: Zero or more messages describing locations that failed to
+      respond.
     jobs: A subset of the requested job information.
     nextPageToken: Set if there may be more results than fit in this response.
   """
 
-  jobs = _messages.MessageField('Job', 1, repeated=True)
-  nextPageToken = _messages.StringField(2)
+  failedLocation = _messages.MessageField('FailedLocation', 1, repeated=True)
+  jobs = _messages.MessageField('Job', 2, repeated=True)
+  nextPageToken = _messages.StringField(3)
 
 
 class MapTask(_messages.Message):
@@ -1760,6 +2066,7 @@ class NameAndKind(_messages.Message):
       AND: Aggregated value represents the logical 'and' of all contributed
         values.
       SET: Aggregated value is a set of unique contributed values.
+      DISTRIBUTION: Aggregated value captures statistics about a distribution.
     """
     INVALID = 0
     SUM = 1
@@ -1769,22 +2076,24 @@ class NameAndKind(_messages.Message):
     OR = 5
     AND = 6
     SET = 7
+    DISTRIBUTION = 8
 
   kind = _messages.EnumField('KindValueValuesEnum', 1)
   name = _messages.StringField(2)
 
 
 class Package(_messages.Message):
-  """Packages that need to be installed in order for a worker to run the steps
-  of the Dataflow job which will be assigned to its worker pool.  This is the
-  mechanism by which the SDK causes code to be loaded onto the workers.  For
-  example, the Dataflow Java SDK might use this to install jars containing the
-  user's code and all of the various dependencies (libraries, data files, etc)
-  required in order for that code to run.
+  """The packages that must be installed in order for a worker to run the
+  steps of the Cloud Dataflow job that will be assigned to its worker pool.
+  This is the mechanism by which the Cloud Dataflow SDK causes code to be
+  loaded onto the workers. For example, the Cloud Dataflow Java SDK might use
+  this to install jars containing the user's code and all of the various
+  dependencies (libraries, data files, etc.) required in order for that code
+  to run.
 
   Fields:
-    location: The resource to read the package from.  The supported resource
-      type is:  Google Cloud Storage:   storage.googleapis.com/{bucket}
+    location: The resource to read the package from. The supported resource
+      type is:  Google Cloud Storage:    storage.googleapis.com/{bucket}
       bucket.storage.googleapis.com/
     name: The name of the package.
   """
@@ -1991,6 +2300,8 @@ class PubsubLocation(_messages.Message):
       /<project-id>/<topic-name>"
     trackingSubscription: If set, specifies the pubsub subscription that will
       be used for tracking custom time timestamps for watermark estimation.
+    withAttributes: If true, then the client has requested to get pubsub
+      attributes.
   """
 
   dropLateData = _messages.BooleanField(1)
@@ -1999,6 +2310,7 @@ class PubsubLocation(_messages.Message):
   timestampLabel = _messages.StringField(4)
   topic = _messages.StringField(5)
   trackingSubscription = _messages.StringField(6)
+  withAttributes = _messages.BooleanField(7)
 
 
 class ReadInstruction(_messages.Message):
@@ -2016,6 +2328,7 @@ class ReportWorkItemStatusRequest(_messages.Message):
 
   Fields:
     currentWorkerTime: The current timestamp at the worker.
+    location: The location which contains the WorkItem's job.
     workItemStatuses: The order is unimportant, except that the order of the
       WorkItemServiceState messages in the ReportWorkItemStatusResponse
       corresponds to the order of WorkItemStatus messages here.
@@ -2026,8 +2339,9 @@ class ReportWorkItemStatusRequest(_messages.Message):
   """
 
   currentWorkerTime = _messages.StringField(1)
-  workItemStatuses = _messages.MessageField('WorkItemStatus', 2, repeated=True)
-  workerId = _messages.StringField(3)
+  location = _messages.StringField(2)
+  workItemStatuses = _messages.MessageField('WorkItemStatus', 3, repeated=True)
+  workerId = _messages.StringField(4)
 
 
 class ReportWorkItemStatusResponse(_messages.Message):
@@ -2060,6 +2374,78 @@ class ReportedParallelism(_messages.Message):
   value = _messages.FloatField(2)
 
 
+class ResourceUtilizationReport(_messages.Message):
+  """Worker metrics exported from workers. This contains resource utilization
+  metrics accumulated from a variety of sources. For more information, see go
+  /df-resource-signals.  Note that this proto closely follows the structure of
+  its DFE siblings in its contents.
+
+  Messages:
+    MetricsValueListEntry: A MetricsValueListEntry object.
+
+  Fields:
+    metrics: Each Struct must parallel DFE worker metrics protos (eg.,
+      cpu_time metric will have nested values \u201ctimestamp_ms, total_ms, rate\u201d).
+  """
+
+  @encoding.MapUnrecognizedFields('additionalProperties')
+  class MetricsValueListEntry(_messages.Message):
+    """A MetricsValueListEntry object.
+
+    Messages:
+      AdditionalProperty: An additional property for a MetricsValueListEntry
+        object.
+
+    Fields:
+      additionalProperties: Properties of the object.
+    """
+
+    class AdditionalProperty(_messages.Message):
+      """An additional property for a MetricsValueListEntry object.
+
+      Fields:
+        key: Name of the additional property.
+        value: A extra_types.JsonValue attribute.
+      """
+
+      key = _messages.StringField(1)
+      value = _messages.MessageField('extra_types.JsonValue', 2)
+
+    additionalProperties = _messages.MessageField('AdditionalProperty', 1, repeated=True)
+
+  metrics = _messages.MessageField('MetricsValueListEntry', 1, repeated=True)
+
+
+class ResourceUtilizationReportResponse(_messages.Message):
+  """Service-side response to WorkerMessage reporting resource utilization.
+  """
+
+
+
+class RuntimeEnvironment(_messages.Message):
+  """The environment values to set at runtime.
+
+  Fields:
+    bypassTempDirValidation: Whether to bypass the safety checks for the job's
+      temporary directory. Use with caution.
+    maxWorkers: The maximum number of Google Compute Engine instances to be
+      made available to your pipeline during execution, from 1 to 1000.
+    serviceAccountEmail: The email address of the service account to run the
+      job as.
+    tempLocation: The Cloud Storage path to use for temporary files. Must be a
+      valid Cloud Storage URL, beginning with `gs://`.
+    zone: The Compute Engine [availability
+      zone](https://cloud.google.com/compute/docs/regions-zones/regions-zones)
+      for launching worker instances to run your pipeline.
+  """
+
+  bypassTempDirValidation = _messages.BooleanField(1)
+  maxWorkers = _messages.IntegerField(2, variant=_messages.Variant.INT32)
+  serviceAccountEmail = _messages.StringField(3)
+  tempLocation = _messages.StringField(4)
+  zone = _messages.StringField(5)
+
+
 class SendDebugCaptureRequest(_messages.Message):
   """Request to send encoded debug information.
 
@@ -2768,7 +3154,7 @@ class Status(_messages.Message):
 
 
 class Step(_messages.Message):
-  """Defines a particular step within a Dataflow job.  A job consists of
+  """Defines a particular step within a Cloud Dataflow job.  A job consists of
   multiple steps, each of which performs some specific operation as part of
   the overall job.  Data is typically passed from one step to another as part
   of the job.  Here's an example of a sequence of steps which together
@@ -2778,25 +3164,25 @@ class Step(_messages.Message):
   extract an element-specific key value.    * Group elements with the same key
   into a single element with     that key, transforming a multiply-keyed
   collection into a     uniquely-keyed collection.    * Write the elements out
-  to some data sink.  (Note that the Dataflow service may be used to run many
-  different types of jobs, not just Map-Reduce).
+  to some data sink.  Note that the Cloud Dataflow service may be used to run
+  many different types of jobs, not just Map-Reduce.
 
   Messages:
-    PropertiesValue: Named properties associated with the step.  Each kind of
+    PropertiesValue: Named properties associated with the step. Each kind of
       predefined step has its own required set of properties.
 
   Fields:
-    kind: The kind of step in the dataflow Job.
-    name: Name identifying the step. This must be unique for each step with
-      respect to all other steps in the dataflow Job.
-    properties: Named properties associated with the step.  Each kind of
+    kind: The kind of step in the Cloud Dataflow job.
+    name: The name that identifies the step. This must be unique for each step
+      with respect to all other steps in the Cloud Dataflow job.
+    properties: Named properties associated with the step. Each kind of
       predefined step has its own required set of properties.
   """
 
   @encoding.MapUnrecognizedFields('additionalProperties')
   class PropertiesValue(_messages.Message):
-    """Named properties associated with the step.  Each kind of predefined
-    step has its own required set of properties.
+    """Named properties associated with the step. Each kind of predefined step
+    has its own required set of properties.
 
     Messages:
       AdditionalProperty: An additional property for a PropertiesValue object.
@@ -3005,30 +3391,32 @@ class TaskRunnerSettings(_messages.Message):
   """Taskrunner configuration settings.
 
   Fields:
-    alsologtostderr: Also send taskrunner log info to stderr?
-    baseTaskDir: Location on the worker for task-specific subdirectories.
+    alsologtostderr: Whether to also send taskrunner log info to stderr.
+    baseTaskDir: The location on the worker for task-specific subdirectories.
     baseUrl: The base URL for the taskrunner to use when accessing Google
       Cloud APIs.  When workers access Google Cloud APIs, they logically do so
       via relative URLs.  If this field is specified, it supplies the base URL
       to use for resolving these relative URLs.  The normative algorithm used
       is defined by RFC 1808, "Relative Uniform Resource Locators".  If not
       specified, the default value is "http://www.googleapis.com/"
-    commandlinesFileName: Store preprocessing commands in this file.
-    continueOnException: Do we continue taskrunner if an exception is hit?
-    dataflowApiVersion: API version of endpoint, e.g. "v1b3"
-    harnessCommand: Command to launch the worker harness.
-    languageHint: Suggested backend language.
-    logDir: Directory on the VM to store logs.
-    logToSerialconsole: Send taskrunner log into to Google Compute Engine VM
-      serial console?
+    commandlinesFileName: The file to store preprocessing commands in.
+    continueOnException: Whether to continue taskrunner if an exception is
+      hit.
+    dataflowApiVersion: The API version of endpoint, e.g. "v1b3"
+    harnessCommand: The command to launch the worker harness.
+    languageHint: The suggested backend language.
+    logDir: The directory on the VM to store logs.
+    logToSerialconsole: Whether to send taskrunner log info to Google Compute
+      Engine VM serial console.
     logUploadLocation: Indicates where to put logs.  If this is not specified,
       the logs will not be uploaded.  The supported resource type is:  Google
       Cloud Storage:   storage.googleapis.com/{bucket}/{object}
       bucket.storage.googleapis.com/{object}
-    oauthScopes: OAuth2 scopes to be requested by the taskrunner in order to
-      access the dataflow API.
-    parallelWorkerSettings: Settings to pass to the parallel worker harness.
-    streamingWorkerMainClass: Streaming worker main class name.
+    oauthScopes: The OAuth2 scopes to be requested by the taskrunner in order
+      to access the Cloud Dataflow API.
+    parallelWorkerSettings: The settings to pass to the parallel worker
+      harness.
+    streamingWorkerMainClass: The streaming worker main class name.
     taskGroup: The UNIX group ID on the worker VM to use for tasks launched by
       taskrunner; e.g. "wheel".
     taskUser: The UNIX user ID on the worker VM to use for tasks launched by
@@ -3037,8 +3425,8 @@ class TaskRunnerSettings(_messages.Message):
       for temporary storage.  The supported resource type is:  Google Cloud
       Storage:   storage.googleapis.com/{bucket}/{object}
       bucket.storage.googleapis.com/{object}
-    vmId: ID string of VM.
-    workflowFileName: Store the workflow in this file.
+    vmId: The ID string of the VM.
+    workflowFileName: The file to store the workflow in.
   """
 
   alsologtostderr = _messages.BooleanField(1)
@@ -3380,6 +3768,7 @@ class WorkerMessage(_messages.Message):
     time: The timestamp of the worker_message.
     workerHealthReport: The health of a worker.
     workerMessageCode: A worker message code.
+    workerMetrics: Resource metrics reported by workers.
   """
 
   @encoding.MapUnrecognizedFields('additionalProperties')
@@ -3415,6 +3804,7 @@ class WorkerMessage(_messages.Message):
   time = _messages.StringField(2)
   workerHealthReport = _messages.MessageField('WorkerHealthReport', 3)
   workerMessageCode = _messages.MessageField('WorkerMessageCode', 4)
+  workerMetrics = _messages.MessageField('ResourceUtilizationReport', 5)
 
 
 class WorkerMessageCode(_messages.Message):
@@ -3509,16 +3899,20 @@ class WorkerMessageResponse(_messages.Message):
   Fields:
     workerHealthReportResponse: The service's response to a worker's health
       report.
+    workerMetricsResponse: Service's response to reporting worker metrics
+      (currently empty).
   """
 
   workerHealthReportResponse = _messages.MessageField('WorkerHealthReportResponse', 1)
+  workerMetricsResponse = _messages.MessageField('ResourceUtilizationReportResponse', 2)
 
 
 class WorkerPool(_messages.Message):
-  """Describes one particular pool of Dataflow workers to be instantiated by
-  the Dataflow service in order to perform the computations required by a job.
-  Note that a workflow job may use multiple pools, in order to match the
-  various computational requirements of the various stages of the job.
+  """Describes one particular pool of Cloud Dataflow workers to be
+  instantiated by the Cloud Dataflow service in order to perform the
+  computations required by a job.  Note that a workflow job may use multiple
+  pools, in order to match the various computational requirements of the
+  various stages of the job.
 
   Enums:
     DefaultPackageSetValueValuesEnum: The default package set to install.
@@ -3526,15 +3920,15 @@ class WorkerPool(_messages.Message):
       useful to worker harnesses written in a particular language.
     IpConfigurationValueValuesEnum: Configuration for VM IPs.
     TeardownPolicyValueValuesEnum: Sets the policy for determining when to
-      turndown worker pool. Allowed values are: TEARDOWN_ALWAYS,
-      TEARDOWN_ON_SUCCESS, and TEARDOWN_NEVER. TEARDOWN_ALWAYS means workers
-      are always torn down regardless of whether the job succeeds.
-      TEARDOWN_ON_SUCCESS means workers are torn down if the job succeeds.
-      TEARDOWN_NEVER means the workers are never torn down.  If the workers
+      turndown worker pool. Allowed values are: `TEARDOWN_ALWAYS`,
+      `TEARDOWN_ON_SUCCESS`, and `TEARDOWN_NEVER`. `TEARDOWN_ALWAYS` means
+      workers are always torn down regardless of whether the job succeeds.
+      `TEARDOWN_ON_SUCCESS` means workers are torn down if the job succeeds.
+      `TEARDOWN_NEVER` means the workers are never torn down.  If the workers
       are not torn down by the service, they will continue to run and use
       Google Compute Engine VM resources in the user's project until they are
       explicitly terminated by the user. Because of this, Google recommends
-      using the TEARDOWN_ALWAYS policy except for small, manually supervised
+      using the `TEARDOWN_ALWAYS` policy except for small, manually supervised
       test jobs.  If unknown or unspecified, the service will attempt to
       choose a reasonable default.
 
@@ -3554,7 +3948,7 @@ class WorkerPool(_messages.Message):
     diskType: Type of root disk for VMs.  If empty or unspecified, the service
       will attempt to choose a reasonable default.
     ipConfiguration: Configuration for VM IPs.
-    kind: The kind of the worker pool; currently only 'harness' and 'shuffle'
+    kind: The kind of the worker pool; currently only `harness` and `shuffle`
       are supported.
     machineType: Machine type (e.g. "n1-standard-1").  If empty or
       unspecified, the service will attempt to choose a reasonable default.
@@ -3578,18 +3972,19 @@ class WorkerPool(_messages.Message):
       workers when using the standard Dataflow task runner.  Users should
       ignore this field.
     teardownPolicy: Sets the policy for determining when to turndown worker
-      pool. Allowed values are: TEARDOWN_ALWAYS, TEARDOWN_ON_SUCCESS, and
-      TEARDOWN_NEVER. TEARDOWN_ALWAYS means workers are always torn down
-      regardless of whether the job succeeds. TEARDOWN_ON_SUCCESS means
-      workers are torn down if the job succeeds. TEARDOWN_NEVER means the
+      pool. Allowed values are: `TEARDOWN_ALWAYS`, `TEARDOWN_ON_SUCCESS`, and
+      `TEARDOWN_NEVER`. `TEARDOWN_ALWAYS` means workers are always torn down
+      regardless of whether the job succeeds. `TEARDOWN_ON_SUCCESS` means
+      workers are torn down if the job succeeds. `TEARDOWN_NEVER` means the
       workers are never torn down.  If the workers are not torn down by the
       service, they will continue to run and use Google Compute Engine VM
       resources in the user's project until they are explicitly terminated by
-      the user. Because of this, Google recommends using the TEARDOWN_ALWAYS
+      the user. Because of this, Google recommends using the `TEARDOWN_ALWAYS`
       policy except for small, manually supervised test jobs.  If unknown or
       unspecified, the service will attempt to choose a reasonable default.
-    workerHarnessContainerImage: Docker container image that executes Dataflow
-      worker harness, residing in Google Container Registry. Required.
+    workerHarnessContainerImage: Required. Docker container image that
+      executes the Cloud Dataflow worker harness, residing in Google Container
+      Registry.
     zone: Zone to run the worker pools in.  If empty or unspecified, the
       service will attempt to choose a reasonable default.
   """
@@ -3628,16 +4023,16 @@ class WorkerPool(_messages.Message):
 
   class TeardownPolicyValueValuesEnum(_messages.Enum):
     """Sets the policy for determining when to turndown worker pool. Allowed
-    values are: TEARDOWN_ALWAYS, TEARDOWN_ON_SUCCESS, and TEARDOWN_NEVER.
-    TEARDOWN_ALWAYS means workers are always torn down regardless of whether
-    the job succeeds. TEARDOWN_ON_SUCCESS means workers are torn down if the
-    job succeeds. TEARDOWN_NEVER means the workers are never torn down.  If
-    the workers are not torn down by the service, they will continue to run
-    and use Google Compute Engine VM resources in the user's project until
-    they are explicitly terminated by the user. Because of this, Google
-    recommends using the TEARDOWN_ALWAYS policy except for small, manually
-    supervised test jobs.  If unknown or unspecified, the service will attempt
-    to choose a reasonable default.
+    values are: `TEARDOWN_ALWAYS`, `TEARDOWN_ON_SUCCESS`, and
+    `TEARDOWN_NEVER`. `TEARDOWN_ALWAYS` means workers are always torn down
+    regardless of whether the job succeeds. `TEARDOWN_ON_SUCCESS` means
+    workers are torn down if the job succeeds. `TEARDOWN_NEVER` means the
+    workers are never torn down.  If the workers are not torn down by the
+    service, they will continue to run and use Google Compute Engine VM
+    resources in the user's project until they are explicitly terminated by
+    the user. Because of this, Google recommends using the `TEARDOWN_ALWAYS`
+    policy except for small, manually supervised test jobs.  If unknown or
+    unspecified, the service will attempt to choose a reasonable default.
 
     Values:
       TEARDOWN_POLICY_UNKNOWN: The teardown policy isn't specified, or is
@@ -3735,16 +4130,16 @@ class WorkerSettings(_messages.Message):
       these relative URLs.  The normative algorithm used is defined by RFC
       1808, "Relative Uniform Resource Locators".  If not specified, the
       default value is "http://www.googleapis.com/"
-    reportingEnabled: Send work progress updates to service.
-    servicePath: The Dataflow service path relative to the root URL, for
+    reportingEnabled: Whether to send work progress updates to the service.
+    servicePath: The Cloud Dataflow service path relative to the root URL, for
       example, "dataflow/v1b3/projects".
     shuffleServicePath: The Shuffle service path relative to the root URL, for
       example, "shuffle/v1beta1".
     tempStoragePrefix: The prefix of the resources the system should use for
       temporary storage.  The supported resource type is:  Google Cloud
-      Storage:   storage.googleapis.com/{bucket}/{object}
+      Storage:    storage.googleapis.com/{bucket}/{object}
       bucket.storage.googleapis.com/{object}
-    workerId: ID of the worker running this pipeline.
+    workerId: The ID of the worker running this pipeline.
   """
 
   baseUrl = _messages.StringField(1)


[47/50] [abbrv] beam git commit: This closes #1871

Posted by da...@apache.org.
This closes #1871


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

Branch: refs/heads/master
Commit: be0e32e36313390ed04106d57f4c9dfeabb91b4d
Parents: 1390699 0b4ee73
Author: Davor Bonaci <da...@google.com>
Authored: Mon Jan 30 13:25:53 2017 -0800
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jan 30 13:25:53 2017 -0800

----------------------------------------------------------------------
 .travis.yml                                          | 15 +++++++++++++++
 .../examples/cookbook/datastore_wordcount.py         |  2 +-
 sdks/python/setup.py                                 |  4 ++--
 3 files changed, 18 insertions(+), 3 deletions(-)
----------------------------------------------------------------------



[13/50] [abbrv] beam git commit: Revert "Remove dataflow_test.py"

Posted by da...@apache.org.
Revert "Remove dataflow_test.py"

This reverts commit d5b90d8383e662e803ea79b31661250a043bcfd2.


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

Branch: refs/heads/master
Commit: 96fcc7d31c2540f867c3a73903c2aa99183a6b8b
Parents: af49908
Author: Robert Bradshaw <ro...@gmail.com>
Authored: Tue Jan 24 09:28:38 2017 -0800
Committer: Robert Bradshaw <ro...@gmail.com>
Committed: Tue Jan 24 09:28:38 2017 -0800

----------------------------------------------------------------------
 sdks/python/apache_beam/dataflow_test.py        | 418 +++++++++++++++++++
 .../apache_beam/transforms/ptransform_test.py   |  67 ---
 .../apache_beam/transforms/sideinputs_test.py   | 208 +--------
 3 files changed, 419 insertions(+), 274 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/96fcc7d3/sdks/python/apache_beam/dataflow_test.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/dataflow_test.py b/sdks/python/apache_beam/dataflow_test.py
new file mode 100644
index 0000000..f410230
--- /dev/null
+++ b/sdks/python/apache_beam/dataflow_test.py
@@ -0,0 +1,418 @@
+#
+# 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.
+#
+
+"""Integration tests for the dataflow package."""
+
+from __future__ import absolute_import
+
+import logging
+import re
+import unittest
+
+import apache_beam as beam
+from apache_beam.pvalue import AsDict
+from apache_beam.pvalue import AsIter as AllOf
+from apache_beam.pvalue import AsList
+from apache_beam.pvalue import AsSingleton
+from apache_beam.pvalue import EmptySideInput
+from apache_beam.pvalue import SideOutputValue
+from apache_beam.test_pipeline import TestPipeline
+from apache_beam.transforms import Create
+from apache_beam.transforms import DoFn
+from apache_beam.transforms import FlatMap
+from apache_beam.transforms import GroupByKey
+from apache_beam.transforms import Map
+from apache_beam.transforms import ParDo
+from apache_beam.transforms import WindowInto
+from apache_beam.transforms.util import assert_that
+from apache_beam.transforms.util import equal_to
+from apache_beam.transforms.window import IntervalWindow
+from apache_beam.transforms.window import WindowFn
+from nose.plugins.attrib import attr
+
+
+class DataflowTest(unittest.TestCase):
+  """Dataflow integration tests."""
+
+  SAMPLE_DATA = ['aa bb cc aa bb aa \n'] * 10
+  SAMPLE_RESULT = [('cc', 10), ('bb', 20), ('aa', 30)]
+
+  @beam.ptransform_fn
+  def Count(pcoll):  # pylint: disable=invalid-name, no-self-argument
+    """A Count transform: v, ... => (v, n), ..."""
+    return (pcoll
+            | 'AddCount' >> Map(lambda x: (x, 1))
+            | 'GroupCounts' >> GroupByKey()
+            | 'AddCounts' >> Map(lambda (x, ones): (x, sum(ones))))
+
+  @attr('ValidatesRunner')
+  def test_word_count(self):
+    pipeline = TestPipeline()
+    lines = pipeline | 'SomeWords' >> Create(DataflowTest.SAMPLE_DATA)
+    result = (
+        (lines | 'GetWords' >> FlatMap(lambda x: re.findall(r'\w+', x)))
+        .apply('CountWords', DataflowTest.Count))
+    assert_that(result, equal_to(DataflowTest.SAMPLE_RESULT))
+    pipeline.run()
+
+  @attr('ValidatesRunner')
+  def test_map(self):
+    pipeline = TestPipeline()
+    lines = pipeline | 'input' >> Create(['a', 'b', 'c'])
+    result = (lines
+              | 'upper' >> Map(str.upper)
+              | 'prefix' >> Map(lambda x, prefix: prefix + x, 'foo-'))
+    assert_that(result, equal_to(['foo-A', 'foo-B', 'foo-C']))
+    pipeline.run()
+
+  @attr('ValidatesRunner')
+  def test_par_do_with_side_input_as_arg(self):
+    pipeline = TestPipeline()
+    words_list = ['aa', 'bb', 'cc']
+    words = pipeline | 'SomeWords' >> Create(words_list)
+    prefix = pipeline | 'SomeString' >> Create(['xyz'])  # side in
+    suffix = 'zyx'
+    result = words | FlatMap(
+        'DecorateWords',
+        lambda x, pfx, sfx: ['%s-%s-%s' % (pfx, x, sfx)],
+        AsSingleton(prefix), suffix)
+    assert_that(result, equal_to(['xyz-%s-zyx' % x for x in words_list]))
+    pipeline.run()
+
+  @attr('ValidatesRunner')
+  def test_par_do_with_side_input_as_keyword_arg(self):
+    pipeline = TestPipeline()
+    words_list = ['aa', 'bb', 'cc']
+    words = pipeline | 'SomeWords' >> Create(words_list)
+    prefix = 'zyx'
+    suffix = pipeline | 'SomeString' >> Create(['xyz'])  # side in
+    result = words | FlatMap(
+        'DecorateWords',
+        lambda x, pfx, sfx: ['%s-%s-%s' % (pfx, x, sfx)],
+        prefix, sfx=AsSingleton(suffix))
+    assert_that(result, equal_to(['zyx-%s-xyz' % x for x in words_list]))
+    pipeline.run()
+
+  @attr('ValidatesRunner')
+  def test_par_do_with_do_fn_object(self):
+    class SomeDoFn(DoFn):
+      """A custom DoFn for a FlatMap transform."""
+
+      def process(self, context, prefix, suffix):
+        return ['%s-%s-%s' % (prefix, context.element, suffix)]
+
+    pipeline = TestPipeline()
+    words_list = ['aa', 'bb', 'cc']
+    words = pipeline | 'SomeWords' >> Create(words_list)
+    prefix = 'zyx'
+    suffix = pipeline | 'SomeString' >> Create(['xyz'])  # side in
+    result = words | 'DecorateWordsDoFn' >> ParDo(
+        SomeDoFn(), prefix, suffix=AsSingleton(suffix))
+    assert_that(result, equal_to(['zyx-%s-xyz' % x for x in words_list]))
+    pipeline.run()
+
+  @attr('ValidatesRunner')
+  def test_par_do_with_multiple_outputs_and_using_yield(self):
+    class SomeDoFn(DoFn):
+      """A custom DoFn using yield."""
+
+      def process(self, context):
+        yield context.element
+        if context.element % 2 == 0:
+          yield SideOutputValue('even', context.element)
+        else:
+          yield SideOutputValue('odd', context.element)
+
+    pipeline = TestPipeline()
+    nums = pipeline | 'Some Numbers' >> Create([1, 2, 3, 4])
+    results = nums | ParDo(
+        'ClassifyNumbers', SomeDoFn()).with_outputs('odd', 'even', main='main')
+    assert_that(results.main, equal_to([1, 2, 3, 4]))
+    assert_that(results.odd, equal_to([1, 3]), label='assert:odd')
+    assert_that(results.even, equal_to([2, 4]), label='assert:even')
+    pipeline.run()
+
+  @attr('ValidatesRunner')
+  def test_par_do_with_multiple_outputs_and_using_return(self):
+    def some_fn(v):
+      if v % 2 == 0:
+        return [v, SideOutputValue('even', v)]
+      else:
+        return [v, SideOutputValue('odd', v)]
+
+    pipeline = TestPipeline()
+    nums = pipeline | 'Some Numbers' >> Create([1, 2, 3, 4])
+    results = nums | FlatMap(
+        'ClassifyNumbers', some_fn).with_outputs('odd', 'even', main='main')
+    assert_that(results.main, equal_to([1, 2, 3, 4]))
+    assert_that(results.odd, equal_to([1, 3]), label='assert:odd')
+    assert_that(results.even, equal_to([2, 4]), label='assert:even')
+    pipeline.run()
+
+  @attr('ValidatesRunner')
+  def test_empty_singleton_side_input(self):
+    pipeline = TestPipeline()
+    pcol = pipeline | 'start' >> Create([1, 2])
+    side = pipeline | 'side' >> Create([])  # Empty side input.
+
+    def my_fn(k, s):
+      v = ('empty' if isinstance(s, EmptySideInput) else 'full')
+      return [(k, v)]
+    result = pcol | 'compute' >> FlatMap(my_fn, AsSingleton(side))
+    assert_that(result, equal_to([(1, 'empty'), (2, 'empty')]))
+    pipeline.run()
+
+  # @attr('ValidatesRunner')
+  # TODO(BEAM-1124): Temporarily disable it due to test failed running on
+  # Dataflow service.
+  def test_multi_valued_singleton_side_input(self):
+    pipeline = TestPipeline()
+    pcol = pipeline | 'start' >> Create([1, 2])
+    side = pipeline | 'side' >> Create([3, 4])  # 2 values in side input.
+    pcol | 'compute' >> FlatMap(lambda x, s: [x * s], AsSingleton(side))  # pylint: disable=expression-not-assigned
+    with self.assertRaises(ValueError):
+      pipeline.run()
+
+  @attr('ValidatesRunner')
+  def test_default_value_singleton_side_input(self):
+    pipeline = TestPipeline()
+    pcol = pipeline | 'start' >> Create([1, 2])
+    side = pipeline | 'side' >> Create([])  # 0 values in side input.
+    result = pcol | FlatMap(lambda x, s: [x * s], AsSingleton(side, 10))
+    assert_that(result, equal_to([10, 20]))
+    pipeline.run()
+
+  @attr('ValidatesRunner')
+  def test_iterable_side_input(self):
+    pipeline = TestPipeline()
+    pcol = pipeline | 'start' >> Create([1, 2])
+    side = pipeline | 'side' >> Create([3, 4])  # 2 values in side input.
+    result = pcol | FlatMap('compute',
+                            lambda x, s: [x * y for y in s], AllOf(side))
+    assert_that(result, equal_to([3, 4, 6, 8]))
+    pipeline.run()
+
+  @attr('ValidatesRunner')
+  def test_undeclared_side_outputs(self):
+    pipeline = TestPipeline()
+    nums = pipeline | 'Some Numbers' >> Create([1, 2, 3, 4])
+    results = nums | FlatMap(
+        'ClassifyNumbers',
+        lambda x: [x, SideOutputValue('even' if x % 2 == 0 else 'odd', x)]
+    ).with_outputs()
+    assert_that(results[None], equal_to([1, 2, 3, 4]))
+    assert_that(results.odd, equal_to([1, 3]), label='assert:odd')
+    assert_that(results.even, equal_to([2, 4]), label='assert:even')
+    pipeline.run()
+
+  @attr('ValidatesRunner')
+  def test_empty_side_outputs(self):
+    pipeline = TestPipeline()
+    nums = pipeline | 'Some Numbers' >> Create([1, 3, 5])
+    results = nums | FlatMap(
+        'ClassifyNumbers',
+        lambda x: [x, SideOutputValue('even' if x % 2 == 0 else 'odd', x)]
+    ).with_outputs()
+    assert_that(results[None], equal_to([1, 3, 5]))
+    assert_that(results.odd, equal_to([1, 3, 5]), label='assert:odd')
+    assert_that(results.even, equal_to([]), label='assert:even')
+    pipeline.run()
+
+  @attr('ValidatesRunner')
+  def test_as_list_and_as_dict_side_inputs(self):
+    a_list = [5, 1, 3, 2, 9]
+    some_pairs = [('crouton', 17), ('supreme', None)]
+    pipeline = TestPipeline()
+    main_input = pipeline | 'main input' >> Create([1])
+    side_list = pipeline | 'side list' >> Create(a_list)
+    side_pairs = pipeline | 'side pairs' >> Create(some_pairs)
+    results = main_input | FlatMap(
+        'concatenate',
+        lambda x, the_list, the_dict: [[x, the_list, the_dict]],
+        AsList(side_list), AsDict(side_pairs))
+
+    def  matcher(expected_elem, expected_list, expected_pairs):
+      def match(actual):
+        [[actual_elem, actual_list, actual_dict]] = actual
+        equal_to([expected_elem])([actual_elem])
+        equal_to(expected_list)(actual_list)
+        equal_to(expected_pairs)(actual_dict.iteritems())
+      return match
+
+    assert_that(results, matcher(1, a_list, some_pairs))
+    pipeline.run()
+
+  @attr('ValidatesRunner')
+  def test_as_singleton_without_unique_labels(self):
+    # This should succeed as calling AsSingleton on the same PCollection twice
+    # with the same defaults will return the same PCollectionView.
+    a_list = [2]
+    pipeline = TestPipeline()
+    main_input = pipeline | 'main input' >> Create([1])
+    side_list = pipeline | 'side list' >> Create(a_list)
+    results = main_input | FlatMap(
+        'test',
+        lambda x, s1, s2: [[x, s1, s2]],
+        AsSingleton(side_list), AsSingleton(side_list))
+
+    def  matcher(expected_elem, expected_singleton):
+      def match(actual):
+        [[actual_elem, actual_singleton1, actual_singleton2]] = actual
+        equal_to([expected_elem])([actual_elem])
+        equal_to([expected_singleton])([actual_singleton1])
+        equal_to([expected_singleton])([actual_singleton2])
+      return match
+
+    assert_that(results, matcher(1, 2))
+    pipeline.run()
+
+  @attr('ValidatesRunner')
+  def test_as_singleton_with_different_defaults_without_unique_labels(self):
+    # This should fail as AsSingleton with distinct default values should create
+    # distinct PCollectionViews with the same full_label.
+    a_list = [2]
+    pipeline = TestPipeline()
+    main_input = pipeline | 'main input' >> Create([1])
+    side_list = pipeline | 'side list' >> Create(a_list)
+
+    with self.assertRaises(RuntimeError) as e:
+      _ = main_input | FlatMap(
+          'test',
+          lambda x, s1, s2: [[x, s1, s2]],
+          AsSingleton(side_list), AsSingleton(side_list, default_value=3))
+    self.assertTrue(
+        e.exception.message.startswith(
+            'Transform "ViewAsSingleton(side list.None)" does not have a '
+            'stable unique label.'))
+
+  @attr('ValidatesRunner')
+  def test_as_singleton_with_different_defaults_with_unique_labels(self):
+    a_list = []
+    pipeline = TestPipeline()
+    main_input = pipeline | 'main input' >> Create([1])
+    side_list = pipeline | 'side list' >> Create(a_list)
+    results = main_input | FlatMap(
+        'test',
+        lambda x, s1, s2: [[x, s1, s2]],
+        AsSingleton('si1', side_list, default_value=2),
+        AsSingleton('si2', side_list, default_value=3))
+
+    def  matcher(expected_elem, expected_singleton1, expected_singleton2):
+      def match(actual):
+        [[actual_elem, actual_singleton1, actual_singleton2]] = actual
+        equal_to([expected_elem])([actual_elem])
+        equal_to([expected_singleton1])([actual_singleton1])
+        equal_to([expected_singleton2])([actual_singleton2])
+      return match
+
+    assert_that(results, matcher(1, 2, 3))
+    pipeline.run()
+
+  @attr('ValidatesRunner')
+  def test_as_list_without_unique_labels(self):
+    # This should succeed as calling AsList on the same PCollection twice will
+    # return the same PCollectionView.
+    a_list = [1, 2, 3]
+    pipeline = TestPipeline()
+    main_input = pipeline | 'main input' >> Create([1])
+    side_list = pipeline | 'side list' >> Create(a_list)
+    results = main_input | FlatMap(
+        'test',
+        lambda x, ls1, ls2: [[x, ls1, ls2]],
+        AsList(side_list), AsList(side_list))
+
+    def  matcher(expected_elem, expected_list):
+      def match(actual):
+        [[actual_elem, actual_list1, actual_list2]] = actual
+        equal_to([expected_elem])([actual_elem])
+        equal_to(expected_list)(actual_list1)
+        equal_to(expected_list)(actual_list2)
+      return match
+
+    assert_that(results, matcher(1, [1, 2, 3]))
+    pipeline.run()
+
+  @attr('ValidatesRunner')
+  def test_as_list_with_unique_labels(self):
+    a_list = [1, 2, 3]
+    pipeline = TestPipeline()
+    main_input = pipeline | 'main input' >> Create([1])
+    side_list = pipeline | 'side list' >> Create(a_list)
+    results = main_input | FlatMap(
+        'test',
+        lambda x, ls1, ls2: [[x, ls1, ls2]],
+        AsList(side_list), AsList(side_list, label='label'))
+
+    def  matcher(expected_elem, expected_list):
+      def match(actual):
+        [[actual_elem, actual_list1, actual_list2]] = actual
+        equal_to([expected_elem])([actual_elem])
+        equal_to(expected_list)(actual_list1)
+        equal_to(expected_list)(actual_list2)
+      return match
+
+    assert_that(results, matcher(1, [1, 2, 3]))
+    pipeline.run()
+
+  @attr('ValidatesRunner')
+  def test_as_dict_with_unique_labels(self):
+    some_kvs = [('a', 1), ('b', 2)]
+    pipeline = TestPipeline()
+    main_input = pipeline | 'main input' >> Create([1])
+    side_kvs = pipeline | 'side kvs' >> Create(some_kvs)
+    results = main_input | FlatMap(
+        'test',
+        lambda x, dct1, dct2: [[x, dct1, dct2]],
+        AsDict(side_kvs), AsDict(side_kvs, label='label'))
+
+    def  matcher(expected_elem, expected_kvs):
+      def match(actual):
+        [[actual_elem, actual_dict1, actual_dict2]] = actual
+        equal_to([expected_elem])([actual_elem])
+        equal_to(expected_kvs)(actual_dict1.iteritems())
+        equal_to(expected_kvs)(actual_dict2.iteritems())
+      return match
+
+    assert_that(results, matcher(1, some_kvs))
+    pipeline.run()
+
+  @attr('ValidatesRunner')
+  def test_window_transform(self):
+    class TestWindowFn(WindowFn):
+      """Windowing function adding two disjoint windows to each element."""
+
+      def assign(self, assign_context):
+        _ = assign_context
+        return [IntervalWindow(10, 20), IntervalWindow(20, 30)]
+
+      def merge(self, existing_windows):
+        return existing_windows
+
+    pipeline = TestPipeline()
+    numbers = pipeline | 'KVs' >> Create([(1, 10), (2, 20), (3, 30)])
+    result = (numbers
+              | 'W' >> WindowInto(windowfn=TestWindowFn())
+              | 'G' >> GroupByKey())
+    assert_that(
+        result, equal_to([(1, [10]), (1, [10]), (2, [20]),
+                          (2, [20]), (3, [30]), (3, [30])]))
+    pipeline.run()
+
+
+if __name__ == '__main__':
+  logging.getLogger().setLevel(logging.INFO)
+  unittest.main()

http://git-wip-us.apache.org/repos/asf/beam/blob/96fcc7d3/sdks/python/apache_beam/transforms/ptransform_test.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/transforms/ptransform_test.py b/sdks/python/apache_beam/transforms/ptransform_test.py
index 68e4482..827bc83 100644
--- a/sdks/python/apache_beam/transforms/ptransform_test.py
+++ b/sdks/python/apache_beam/transforms/ptransform_test.py
@@ -24,7 +24,6 @@ import re
 import unittest
 
 import hamcrest as hc
-from nose.plugins.attrib import attr
 
 import apache_beam as beam
 from apache_beam.test_pipeline import TestPipeline
@@ -190,72 +189,6 @@ class PTransformTest(unittest.TestCase):
     assert_that(r2.m, equal_to([3, 4, 5]), label='r2')
     pipeline.run()
 
-  @attr('ValidatesRunner')
-  def test_par_do_with_multiple_outputs_and_using_yield(self):
-    class SomeDoFn(beam.DoFn):
-      """A custom DoFn using yield."""
-
-      def process(self, context):
-        yield context.element
-        if context.element % 2 == 0:
-          yield pvalue.SideOutputValue('even', context.element)
-        else:
-          yield pvalue.SideOutputValue('odd', context.element)
-
-    pipeline = TestPipeline()
-    nums = pipeline | 'Some Numbers' >> beam.Create([1, 2, 3, 4])
-    results = nums | beam.ParDo(
-        'ClassifyNumbers', SomeDoFn()).with_outputs('odd', 'even', main='main')
-    assert_that(results.main, equal_to([1, 2, 3, 4]))
-    assert_that(results.odd, equal_to([1, 3]), label='assert:odd')
-    assert_that(results.even, equal_to([2, 4]), label='assert:even')
-    pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_par_do_with_multiple_outputs_and_using_return(self):
-    def some_fn(v):
-      if v % 2 == 0:
-        return [v, pvalue.SideOutputValue('even', v)]
-      else:
-        return [v, pvalue.SideOutputValue('odd', v)]
-
-    pipeline = TestPipeline()
-    nums = pipeline | 'Some Numbers' >> beam.Create([1, 2, 3, 4])
-    results = nums | beam.FlatMap(
-        'ClassifyNumbers', some_fn).with_outputs('odd', 'even', main='main')
-    assert_that(results.main, equal_to([1, 2, 3, 4]))
-    assert_that(results.odd, equal_to([1, 3]), label='assert:odd')
-    assert_that(results.even, equal_to([2, 4]), label='assert:even')
-    pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_undeclared_side_outputs(self):
-    pipeline = TestPipeline()
-    nums = pipeline | 'Some Numbers' >> beam.Create([1, 2, 3, 4])
-    results = nums | beam.FlatMap(
-        'ClassifyNumbers',
-        lambda x: [x,
-                   pvalue.SideOutputValue('even' if x % 2 == 0 else 'odd', x)]
-    ).with_outputs()
-    assert_that(results[None], equal_to([1, 2, 3, 4]))
-    assert_that(results.odd, equal_to([1, 3]), label='assert:odd')
-    assert_that(results.even, equal_to([2, 4]), label='assert:even')
-    pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_empty_side_outputs(self):
-    pipeline = TestPipeline()
-    nums = pipeline | 'Some Numbers' >> beam.Create([1, 3, 5])
-    results = nums | beam.FlatMap(
-        'ClassifyNumbers',
-        lambda x: [x,
-                   pvalue.SideOutputValue('even' if x % 2 == 0 else 'odd', x)]
-    ).with_outputs()
-    assert_that(results[None], equal_to([1, 3, 5]))
-    assert_that(results.odd, equal_to([1, 3, 5]), label='assert:odd')
-    assert_that(results.even, equal_to([]), label='assert:even')
-    pipeline.run()
-
   def test_do_requires_do_fn_returning_iterable(self):
     # This function is incorrect because it returns an object that isn't an
     # iterable.

http://git-wip-us.apache.org/repos/asf/beam/blob/96fcc7d3/sdks/python/apache_beam/transforms/sideinputs_test.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/transforms/sideinputs_test.py b/sdks/python/apache_beam/transforms/sideinputs_test.py
index 9384e7b..a327dc8 100644
--- a/sdks/python/apache_beam/transforms/sideinputs_test.py
+++ b/sdks/python/apache_beam/transforms/sideinputs_test.py
@@ -20,10 +20,7 @@
 import logging
 import unittest
 
-from nose.plugins.attrib import attr
-
 import apache_beam as beam
-from apache_beam.test_pipeline import TestPipeline
 from apache_beam.transforms import window
 from apache_beam.transforms.util import assert_that, equal_to
 
@@ -31,7 +28,7 @@ from apache_beam.transforms.util import assert_that, equal_to
 class SideInputsTest(unittest.TestCase):
 
   def create_pipeline(self):
-    return TestPipeline('DirectRunner')
+    return beam.Pipeline('DirectRunner')
 
   def run_windowed_side_inputs(self, elements, main_window_fn,
                                side_window_fn=None,
@@ -128,209 +125,6 @@ class SideInputsTest(unittest.TestCase):
             (11, {'k11': 'v11'}),
         ])
 
-  @attr('ValidatesRunner')
-  def test_empty_singleton_side_input(self):
-    pipeline = self.create_pipeline()
-    pcol = pipeline | 'start' >> beam.Create([1, 2])
-    side = pipeline | 'side' >> beam.Create([])  # Empty side input.
-
-    def my_fn(k, s):
-      # TODO(robertwb): Should this be an error as in Java?
-      v = ('empty' if isinstance(s, beam.pvalue.EmptySideInput) else 'full')
-      return [(k, v)]
-    result = pcol | 'compute' >> beam.FlatMap(
-        my_fn, beam.pvalue.AsSingleton(side))
-    assert_that(result, equal_to([(1, 'empty'), (2, 'empty')]))
-    pipeline.run()
-
-  # @attr('ValidatesRunner')
-  # TODO(BEAM-1124): Temporarily disable it due to test failed running on
-  # Dataflow service.
-  def test_multi_valued_singleton_side_input(self):
-    pipeline = self.create_pipeline()
-    pcol = pipeline | 'start' >> beam.Create([1, 2])
-    side = pipeline | 'side' >> beam.Create([3, 4])  # 2 values in side input.
-    pcol | 'compute' >> beam.FlatMap(  # pylint: disable=expression-not-assigned
-        lambda x, s: [x * s], beam.pvalue.AsSingleton(side))
-    with self.assertRaises(ValueError):
-      pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_default_value_singleton_side_input(self):
-    pipeline = self.create_pipeline()
-    pcol = pipeline | 'start' >> beam.Create([1, 2])
-    side = pipeline | 'side' >> beam.Create([])  # 0 values in side input.
-    result = pcol | beam.FlatMap(
-        lambda x, s: [x * s], beam.pvalue.AsSingleton(side, 10))
-    assert_that(result, equal_to([10, 20]))
-    pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_iterable_side_input(self):
-    pipeline = self.create_pipeline()
-    pcol = pipeline | 'start' >> beam.Create([1, 2])
-    side = pipeline | 'side' >> beam.Create([3, 4])  # 2 values in side input.
-    result = pcol | 'compute' >> beam.FlatMap(
-        lambda x, s: [x * y for y in s],
-        beam.pvalue.AsIter(side))
-    assert_that(result, equal_to([3, 4, 6, 8]))
-    pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_as_list_and_as_dict_side_inputs(self):
-    a_list = [5, 1, 3, 2, 9]
-    some_pairs = [('crouton', 17), ('supreme', None)]
-    pipeline = self.create_pipeline()
-    main_input = pipeline | 'main input' >> beam.Create([1])
-    side_list = pipeline | 'side list' >> beam.Create(a_list)
-    side_pairs = pipeline | 'side pairs' >> beam.Create(some_pairs)
-    results = main_input | 'concatenate' >> beam.FlatMap(
-        lambda x, the_list, the_dict: [[x, the_list, the_dict]],
-        beam.pvalue.AsList(side_list), beam.pvalue.AsDict(side_pairs))
-
-    def  matcher(expected_elem, expected_list, expected_pairs):
-      def match(actual):
-        [[actual_elem, actual_list, actual_dict]] = actual
-        equal_to([expected_elem])([actual_elem])
-        equal_to(expected_list)(actual_list)
-        equal_to(expected_pairs)(actual_dict.iteritems())
-      return match
-
-    assert_that(results, matcher(1, a_list, some_pairs))
-    pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_as_singleton_without_unique_labels(self):
-    # This should succeed as calling beam.pvalue.AsSingleton on the same
-    # PCollection twice with the same defaults will return the same
-    # PCollectionView.
-    a_list = [2]
-    pipeline = self.create_pipeline()
-    main_input = pipeline | 'main input' >> beam.Create([1])
-    side_list = pipeline | 'side list' >> beam.Create(a_list)
-    results = main_input | beam.FlatMap(
-        lambda x, s1, s2: [[x, s1, s2]],
-        beam.pvalue.AsSingleton(side_list), beam.pvalue.AsSingleton(side_list))
-
-    def  matcher(expected_elem, expected_singleton):
-      def match(actual):
-        [[actual_elem, actual_singleton1, actual_singleton2]] = actual
-        equal_to([expected_elem])([actual_elem])
-        equal_to([expected_singleton])([actual_singleton1])
-        equal_to([expected_singleton])([actual_singleton2])
-      return match
-
-    assert_that(results, matcher(1, 2))
-    pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_as_singleton_with_different_defaults_without_unique_labels(self):
-    # This should fail as beam.pvalue.AsSingleton with distinct default values
-    # should beam.Create distinct PCollectionViews with the same full_label.
-    a_list = [2]
-    pipeline = self.create_pipeline()
-    main_input = pipeline | 'main input' >> beam.Create([1])
-    side_list = pipeline | 'side list' >> beam.Create(a_list)
-
-    with self.assertRaises(RuntimeError) as e:
-      _ = main_input | beam.FlatMap(
-          lambda x, s1, s2: [[x, s1, s2]],
-          beam.pvalue.AsSingleton(side_list),
-          beam.pvalue.AsSingleton(side_list, default_value=3))
-    self.assertTrue(
-        e.exception.message.startswith(
-            'Transform "ViewAsSingleton(side list.None)" does not have a '
-            'stable unique label.'))
-
-  @attr('ValidatesRunner')
-  def test_as_singleton_with_different_defaults_with_unique_labels(self):
-    a_list = []
-    pipeline = self.create_pipeline()
-    main_input = pipeline | 'main input' >> beam.Create([1])
-    side_list = pipeline | 'side list' >> beam.Create(a_list)
-    results = main_input | beam.FlatMap(
-        lambda x, s1, s2: [[x, s1, s2]],
-        beam.pvalue.AsSingleton('si1', side_list, default_value=2),
-        beam.pvalue.AsSingleton('si2', side_list, default_value=3))
-
-    def  matcher(expected_elem, expected_singleton1, expected_singleton2):
-      def match(actual):
-        [[actual_elem, actual_singleton1, actual_singleton2]] = actual
-        equal_to([expected_elem])([actual_elem])
-        equal_to([expected_singleton1])([actual_singleton1])
-        equal_to([expected_singleton2])([actual_singleton2])
-      return match
-
-    assert_that(results, matcher(1, 2, 3))
-    pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_as_list_without_unique_labels(self):
-    # This should succeed as calling beam.pvalue.AsList on the same
-    # PCollection twice will return the same PCollectionView.
-    a_list = [1, 2, 3]
-    pipeline = self.create_pipeline()
-    main_input = pipeline | 'main input' >> beam.Create([1])
-    side_list = pipeline | 'side list' >> beam.Create(a_list)
-    results = main_input | beam.FlatMap(
-        lambda x, ls1, ls2: [[x, ls1, ls2]],
-        beam.pvalue.AsList(side_list), beam.pvalue.AsList(side_list))
-
-    def  matcher(expected_elem, expected_list):
-      def match(actual):
-        [[actual_elem, actual_list1, actual_list2]] = actual
-        equal_to([expected_elem])([actual_elem])
-        equal_to(expected_list)(actual_list1)
-        equal_to(expected_list)(actual_list2)
-      return match
-
-    assert_that(results, matcher(1, [1, 2, 3]))
-    pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_as_list_with_unique_labels(self):
-    a_list = [1, 2, 3]
-    pipeline = self.create_pipeline()
-    main_input = pipeline | 'main input' >> beam.Create([1])
-    side_list = pipeline | 'side list' >> beam.Create(a_list)
-    results = main_input | beam.FlatMap(
-        lambda x, ls1, ls2: [[x, ls1, ls2]],
-        beam.pvalue.AsList(side_list),
-        beam.pvalue.AsList(side_list, label='label'))
-
-    def  matcher(expected_elem, expected_list):
-      def match(actual):
-        [[actual_elem, actual_list1, actual_list2]] = actual
-        equal_to([expected_elem])([actual_elem])
-        equal_to(expected_list)(actual_list1)
-        equal_to(expected_list)(actual_list2)
-      return match
-
-    assert_that(results, matcher(1, [1, 2, 3]))
-    pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_as_dict_with_unique_labels(self):
-    some_kvs = [('a', 1), ('b', 2)]
-    pipeline = self.create_pipeline()
-    main_input = pipeline | 'main input' >> beam.Create([1])
-    side_kvs = pipeline | 'side kvs' >> beam.Create(some_kvs)
-    results = main_input | beam.FlatMap(
-        lambda x, dct1, dct2: [[x, dct1, dct2]],
-        beam.pvalue.AsDict(side_kvs),
-        beam.pvalue.AsDict(side_kvs, label='label'))
-
-    def  matcher(expected_elem, expected_kvs):
-      def match(actual):
-        [[actual_elem, actual_dict1, actual_dict2]] = actual
-        equal_to([expected_elem])([actual_elem])
-        equal_to(expected_kvs)(actual_dict1.iteritems())
-        equal_to(expected_kvs)(actual_dict2.iteritems())
-      return match
-
-    assert_that(results, matcher(1, some_kvs))
-    pipeline.run()
-
 
 if __name__ == '__main__':
   logging.getLogger().setLevel(logging.DEBUG)


[30/50] [abbrv] beam git commit: Closes #1835

Posted by da...@apache.org.
Closes #1835


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

Branch: refs/heads/master
Commit: 3d6f20d677bd7397e6e2d099c829bf4c439f8d18
Parents: e3849af b148f5c
Author: Robert Bradshaw <ro...@google.com>
Authored: Thu Jan 26 15:28:50 2017 -0800
Committer: Robert Bradshaw <ro...@google.com>
Committed: Thu Jan 26 15:28:50 2017 -0800

----------------------------------------------------------------------
 sdks/python/apache_beam/metrics/execution.pxd   | 31 +++++++++
 sdks/python/apache_beam/metrics/execution.py    | 70 ++++++++++++--------
 sdks/python/apache_beam/runners/common.pxd      |  2 +
 sdks/python/apache_beam/runners/common.py       | 11 ++-
 .../apache_beam/runners/direct/executor.py      | 12 ++--
 .../runners/direct/transform_evaluator.py       | 54 ++++++++-------
 sdks/python/setup.py                            |  1 +
 7 files changed, 125 insertions(+), 56 deletions(-)
----------------------------------------------------------------------



[26/50] [abbrv] beam git commit: Closes #1837

Posted by da...@apache.org.
Closes #1837


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

Branch: refs/heads/master
Commit: c6420df9791eb6083fba1f74bd88e06ce8f6a61f
Parents: 4e1028b 2aa7d47
Author: Robert Bradshaw <ro...@google.com>
Authored: Wed Jan 25 16:18:10 2017 -0800
Committer: Robert Bradshaw <ro...@google.com>
Committed: Wed Jan 25 16:18:10 2017 -0800

----------------------------------------------------------------------
 sdks/python/apache_beam/dataflow_test.py        | 418 -------------------
 .../apache_beam/transforms/ptransform_test.py   |  67 +++
 .../apache_beam/transforms/sideinputs_test.py   | 208 ++++++++-
 3 files changed, 274 insertions(+), 419 deletions(-)
----------------------------------------------------------------------



[03/50] [abbrv] beam git commit: Implement Annotation based NewDoFn in python SDK

Posted by da...@apache.org.
Implement Annotation based NewDoFn in python SDK


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

Branch: refs/heads/master
Commit: 9e272ecf639b7b13f23a83868fd101a437159c1c
Parents: 946135f
Author: Sourabh Bajaj <so...@google.com>
Authored: Fri Jan 20 17:17:25 2017 -0800
Committer: Robert Bradshaw <ro...@gmail.com>
Committed: Sat Jan 21 20:37:07 2017 -0800

----------------------------------------------------------------------
 sdks/python/apache_beam/pipeline_test.py        | 100 ++++++++-
 sdks/python/apache_beam/runners/common.pxd      |   4 +
 sdks/python/apache_beam/runners/common.py       | 221 +++++++++++++------
 .../runners/direct/transform_evaluator.py       |  15 +-
 sdks/python/apache_beam/transforms/core.py      | 113 +++++++++-
 sdks/python/apache_beam/typehints/decorators.py |   2 +-
 sdks/python/apache_beam/typehints/typecheck.py  | 145 ++++++++++++
 7 files changed, 531 insertions(+), 69 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/9e272ecf/sdks/python/apache_beam/pipeline_test.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/pipeline_test.py b/sdks/python/apache_beam/pipeline_test.py
index 336bf54..93b68d1 100644
--- a/sdks/python/apache_beam/pipeline_test.py
+++ b/sdks/python/apache_beam/pipeline_test.py
@@ -24,15 +24,23 @@ import unittest
 from apache_beam.pipeline import Pipeline
 from apache_beam.pipeline import PipelineOptions
 from apache_beam.pipeline import PipelineVisitor
+from apache_beam.pvalue import AsSingleton
 from apache_beam.runners.dataflow.native_io.iobase import NativeSource
 from apache_beam.test_pipeline import TestPipeline
 from apache_beam.transforms import CombineGlobally
 from apache_beam.transforms import Create
 from apache_beam.transforms import FlatMap
 from apache_beam.transforms import Map
+from apache_beam.transforms import NewDoFn
+from apache_beam.transforms import ParDo
 from apache_beam.transforms import PTransform
 from apache_beam.transforms import Read
-from apache_beam.transforms.util import assert_that, equal_to
+from apache_beam.transforms import WindowInto
+from apache_beam.transforms.util import assert_that
+from apache_beam.transforms.util import equal_to
+from apache_beam.transforms.window import IntervalWindow
+from apache_beam.transforms.window import WindowFn
+from apache_beam.utils.timestamp import MIN_TIMESTAMP
 
 
 class FakeSource(NativeSource):
@@ -241,6 +249,96 @@ class PipelineTest(unittest.TestCase):
     self.assertEqual([1, 4, 9], p | Create([1, 2, 3]) | Map(lambda x: x*x))
 
 
+class NewDoFnTest(unittest.TestCase):
+
+  def setUp(self):
+    self.runner_name = 'DirectRunner'
+
+  def test_element(self):
+    class TestDoFn(NewDoFn):
+      def process(self, element):
+        yield element + 10
+
+    pipeline = TestPipeline(runner=self.runner_name)
+    pcoll = pipeline | 'Create' >> Create([1, 2]) | 'Do' >> ParDo(TestDoFn())
+    assert_that(pcoll, equal_to([11, 12]))
+    pipeline.run()
+
+  def test_context_param(self):
+    class TestDoFn(NewDoFn):
+      def process(self, element, context=NewDoFn.ContextParam):
+        yield context.element + 10
+
+    pipeline = TestPipeline(runner=self.runner_name)
+    pcoll = pipeline | 'Create' >> Create([1, 2])| 'Do' >> ParDo(TestDoFn())
+    assert_that(pcoll, equal_to([11, 12]))
+    pipeline.run()
+
+  def test_side_input_no_tag(self):
+    class TestDoFn(NewDoFn):
+      def process(self, element, prefix, suffix):
+        return ['%s-%s-%s' % (prefix, element, suffix)]
+
+    pipeline = TestPipeline()
+    words_list = ['aa', 'bb', 'cc']
+    words = pipeline | 'SomeWords' >> Create(words_list)
+    prefix = 'zyx'
+    suffix = pipeline | 'SomeString' >> Create(['xyz'])  # side in
+    result = words | 'DecorateWordsDoFnNoTag' >> ParDo(
+        TestDoFn(), prefix, suffix=AsSingleton(suffix))
+    assert_that(result, equal_to(['zyx-%s-xyz' % x for x in words_list]))
+    pipeline.run()
+
+  def test_side_input_tagged(self):
+    class TestDoFn(NewDoFn):
+      def process(self, element, prefix, suffix=NewDoFn.SideInputParam):
+        return ['%s-%s-%s' % (prefix, element, suffix)]
+
+    pipeline = TestPipeline()
+    words_list = ['aa', 'bb', 'cc']
+    words = pipeline | 'SomeWords' >> Create(words_list)
+    prefix = 'zyx'
+    suffix = pipeline | 'SomeString' >> Create(['xyz'])  # side in
+    result = words | 'DecorateWordsDoFnNoTag' >> ParDo(
+        TestDoFn(), prefix, suffix=AsSingleton(suffix))
+    assert_that(result, equal_to(['zyx-%s-xyz' % x for x in words_list]))
+    pipeline.run()
+
+  def test_window_param(self):
+    class TestDoFn(NewDoFn):
+      def process(self, element, window=NewDoFn.WindowParam):
+        yield (float(window.start), float(window.end))
+
+    class TestWindowFn(WindowFn):
+      """Windowing function adding two disjoint windows to each element."""
+
+      def assign(self, assign_context):
+        _ = assign_context
+        return [IntervalWindow(10, 20), IntervalWindow(20, 30)]
+
+      def merge(self, existing_windows):
+        return existing_windows
+
+    pipeline = TestPipeline(runner=self.runner_name)
+    pcoll = (pipeline
+             | 'KVs' >> Create([(1, 10), (2, 20)])
+             | 'W' >> WindowInto(windowfn=TestWindowFn())
+             | 'Do' >> ParDo(TestDoFn()))
+    assert_that(pcoll, equal_to([(10.0, 20.0), (10.0, 20.0),
+                                 (20.0, 30.0), (20.0, 30.0)]))
+    pipeline.run()
+
+  def test_timestamp_param(self):
+    class TestDoFn(NewDoFn):
+      def process(self, element, timestamp=NewDoFn.TimestampParam):
+        yield timestamp
+
+    pipeline = TestPipeline(runner=self.runner_name)
+    pcoll = pipeline | 'Create' >> Create([1, 2]) | 'Do' >> ParDo(TestDoFn())
+    assert_that(pcoll, equal_to([MIN_TIMESTAMP, MIN_TIMESTAMP]))
+    pipeline.run()
+
+
 class Bacon(PipelineOptions):
 
   @classmethod

http://git-wip-us.apache.org/repos/asf/beam/blob/9e272ecf/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 085fd11..06fe434 100644
--- a/sdks/python/apache_beam/runners/common.pxd
+++ b/sdks/python/apache_beam/runners/common.pxd
@@ -36,6 +36,10 @@ cdef class DoFnRunner(Receiver):
   cdef object tagged_receivers
   cdef LoggingContext logging_context
   cdef object step_name
+  cdef object is_new_dofn
+  cdef object args
+  cdef object kwargs
+  cdef object side_inputs
   cdef bint has_windowed_side_inputs
 
   cdef Receiver main_receivers

http://git-wip-us.apache.org/repos/asf/beam/blob/9e272ecf/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 cc834ba..0f63cbc 100644
--- a/sdks/python/apache_beam/runners/common.py
+++ b/sdks/python/apache_beam/runners/common.py
@@ -71,50 +71,21 @@ class DoFnRunner(Receiver):
                # Preferred alternative to context
                # TODO(robertwb): Remove once all runners are updated.
                state=None):
-    self.has_windowed_side_inputs = False  # Set to True in one case below.
-    if not args and not kwargs:
-      self.dofn = fn
-      self.dofn_process = fn.process
-    else:
-      global_window = window.GlobalWindow()
-      # TODO(robertwb): Remove when all runners pass side input maps.
-      side_inputs = [side_input
-                     if isinstance(side_input, sideinputs.SideInputMap)
-                     else {global_window: side_input}
-                     for side_input in side_inputs]
-      if side_inputs and all(
-          isinstance(side_input, dict) or side_input.is_globally_windowed()
-          for side_input in side_inputs):
-        args, kwargs = util.insert_values_in_args(
-            args, kwargs, [side_input[global_window]
-                           for side_input in side_inputs])
-        side_inputs = []
-      if side_inputs:
-        self.has_windowed_side_inputs = True
-
-        def process(context):
-          w = context.windows[0]
-          cur_args, cur_kwargs = util.insert_values_in_args(
-              args, kwargs, [side_input[w] for side_input in side_inputs])
-          return fn.process(context, *cur_args, **cur_kwargs)
-        self.dofn_process = process
-      elif kwargs:
-        self.dofn_process = lambda context: fn.process(context, *args, **kwargs)
-      else:
-        self.dofn_process = lambda context: fn.process(context, *args)
-
-      class CurriedFn(core.DoFn):
+    self.step_name = step_name
+    self.window_fn = windowing.windowfn
+    self.tagged_receivers = tagged_receivers
 
-        start_bundle = staticmethod(fn.start_bundle)
-        process = staticmethod(self.dofn_process)
-        finish_bundle = staticmethod(fn.finish_bundle)
+    global_window = window.GlobalWindow()
 
-      self.dofn = CurriedFn()
+    if logging_context:
+      self.logging_context = logging_context
+    else:
+      self.logging_context = get_logging_context(logger, step_name=step_name)
 
-    self.window_fn = windowing.windowfn
-    self.tagged_receivers = tagged_receivers
-    self.step_name = step_name
+    # Optimize for the common case.
+    self.main_receivers = as_receiver(tagged_receivers[None])
 
+    # TODO(sourabh): Deprecate the use of context
     if state:
       assert context is None
       self.context = DoFnContext(self.step_name, state=state)
@@ -122,48 +93,172 @@ class DoFnRunner(Receiver):
       assert context is not None
       self.context = context
 
-    if logging_context:
-      self.logging_context = logging_context
+    # TODO(Sourabhbajaj): Remove the usage of OldDoFn
+    if isinstance(fn, core.NewDoFn):
+      self.is_new_dofn = True
+
+      # SideInputs
+      self.side_inputs = [side_input
+                          if isinstance(side_input, sideinputs.SideInputMap)
+                          else {global_window: side_input}
+                          for side_input in side_inputs]
+      self.has_windowed_side_inputs = not all(
+          isinstance(si, dict) or si.is_globally_windowed()
+          for si in self.side_inputs)
+
+      self.args = args if args else []
+      self.kwargs = kwargs if kwargs else {}
+      self.dofn = fn
+
     else:
-      self.logging_context = get_logging_context(logger, step_name=step_name)
+      self.is_new_dofn = False
+      self.has_windowed_side_inputs = False  # Set to True in one case below.
+      if not args and not kwargs:
+        self.dofn = fn
+        self.dofn_process = fn.process
+      else:
+        # TODO(robertwb): Remove when all runners pass side input maps.
+        side_inputs = [side_input
+                       if isinstance(side_input, sideinputs.SideInputMap)
+                       else {global_window: side_input}
+                       for side_input in side_inputs]
+        if side_inputs and all(
+            isinstance(side_input, dict) or side_input.is_globally_windowed()
+            for side_input in side_inputs):
+          args, kwargs = util.insert_values_in_args(
+              args, kwargs, [side_input[global_window]
+                             for side_input in side_inputs])
+          side_inputs = []
+        if side_inputs:
+          self.has_windowed_side_inputs = True
+
+          def process(context):
+            w = context.windows[0]
+            cur_args, cur_kwargs = util.insert_values_in_args(
+                args, kwargs, [side_input[w] for side_input in side_inputs])
+            return fn.process(context, *cur_args, **cur_kwargs)
+          self.dofn_process = process
+        elif kwargs:
+          self.dofn_process = lambda context: fn.process(
+              context, *args, **kwargs)
+        else:
+          self.dofn_process = lambda context: fn.process(context, *args)
 
-    # Optimize for the common case.
-    self.main_receivers = as_receiver(tagged_receivers[None])
+        class CurriedFn(core.DoFn):
+
+          start_bundle = staticmethod(fn.start_bundle)
+          process = staticmethod(self.dofn_process)
+          finish_bundle = staticmethod(fn.finish_bundle)
+
+        self.dofn = CurriedFn()
 
   def receive(self, windowed_value):
     self.process(windowed_value)
 
-  def start(self):
-    self.context.set_element(None)
+  def old_dofn_process(self, element):
+    if self.has_windowed_side_inputs and len(element.windows) > 1:
+      for w in element.windows:
+        self.context.set_element(
+            WindowedValue(element.value, element.timestamp, (w,)))
+        self._process_outputs(element, self.dofn_process(self.context))
+    else:
+      self.context.set_element(element)
+      self._process_outputs(element, self.dofn_process(self.context))
+
+  def new_dofn_process(self, element):
+    self.context.set_element(element)
+    arguments, _, _, defaults = self.dofn.get_function_arguments('process')
+    defaults = defaults if defaults else []
+
+    self_in_args = int(self.dofn.is_process_bounded())
+
+    # Call for the process function for each window if has windowed side inputs
+    # or if the process accesses the window parameter. We can just call it once
+    # otherwise as none of the arguments are changing
+    if self.has_windowed_side_inputs or core.NewDoFn.WindowParam in defaults:
+      windows = element.windows
+    else:
+      windows = [window.GlobalWindow()]
+
+    for w in windows:
+      args, kwargs = util.insert_values_in_args(
+          self.args, self.kwargs,
+          [s[w] for s in self.side_inputs])
+
+      # If there are more arguments than the default then the first argument
+      # should be the element and the rest should be picked from the side
+      # inputs as window and timestamp should always be tagged
+      if len(arguments) > len(defaults) + self_in_args:
+        if core.NewDoFn.ElementParam not in defaults:
+          args_to_pick = len(arguments) - len(defaults) - 1 - self_in_args
+          final_args = [element.value] + args[:args_to_pick]
+        else:
+          args_to_pick = len(arguments) - len(defaults) - self_in_args
+          final_args = args[:args_to_pick]
+      else:
+        args_to_pick = 0
+        final_args = []
+      args = iter(args[args_to_pick:])
+
+      for a, d in zip(arguments[-len(defaults):], defaults):
+        if d == core.NewDoFn.ElementParam:
+          final_args.append(element.value)
+        elif d == core.NewDoFn.ContextParam:
+          final_args.append(self.context)
+        elif d == core.NewDoFn.WindowParam:
+          final_args.append(w)
+        elif d == core.NewDoFn.TimestampParam:
+          final_args.append(element.timestamp)
+        elif d == core.NewDoFn.SideInputParam:
+          # If no more args are present then the value must be passed via kwarg
+          try:
+            final_args.append(args.next())
+          except StopIteration:
+            if a not in kwargs:
+              raise
+        else:
+          # If no more args are present then the value must be passed via kwarg
+          try:
+            final_args.append(args.next())
+          except StopIteration:
+            if a not in kwargs:
+              kwargs[a] = d
+      final_args.extend(list(args))
+      self._process_outputs(element, self.dofn.process(*final_args, **kwargs))
+
+  def _invoke_bundle_method(self, method):
     try:
       self.logging_context.enter()
-      self._process_outputs(None, self.dofn.start_bundle(self.context))
+      self.context.set_element(None)
+      f = getattr(self.dofn, method)
+
+      # TODO(Sourabhbajaj): Remove this if-else
+      if self.is_new_dofn:
+        _, _, _, defaults = self.dofn.get_function_arguments(method)
+        defaults = defaults if defaults else []
+        args = [self.context if d == core.NewDoFn.ContextParam else d
+                for d in defaults]
+        self._process_outputs(None, f(*args))
+      else:
+        self._process_outputs(None, f(self.context))
     except BaseException as exn:
       self.reraise_augmented(exn)
     finally:
       self.logging_context.exit()
 
+  def start(self):
+    self._invoke_bundle_method('start_bundle')
+
   def finish(self):
-    self.context.set_element(None)
-    try:
-      self.logging_context.enter()
-      self._process_outputs(None, self.dofn.finish_bundle(self.context))
-    except BaseException as exn:
-      self.reraise_augmented(exn)
-    finally:
-      self.logging_context.exit()
+    self._invoke_bundle_method('finish_bundle')
 
   def process(self, element):
     try:
       self.logging_context.enter()
-      if self.has_windowed_side_inputs and len(element.windows) > 1:
-        for w in element.windows:
-          self.context.set_element(
-              WindowedValue(element.value, element.timestamp, (w,)))
-          self._process_outputs(element, self.dofn_process(self.context))
+      if self.is_new_dofn:
+        self.new_dofn_process(element)
       else:
-        self.context.set_element(element)
-        self._process_outputs(element, self.dofn_process(self.context))
+        self.old_dofn_process(element)
     except BaseException as exn:
       self.reraise_augmented(exn)
     finally:

http://git-wip-us.apache.org/repos/asf/beam/blob/9e272ecf/sdks/python/apache_beam/runners/direct/transform_evaluator.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/runners/direct/transform_evaluator.py b/sdks/python/apache_beam/runners/direct/transform_evaluator.py
index b4c43ba..ec2b3a1 100644
--- a/sdks/python/apache_beam/runners/direct/transform_evaluator.py
+++ b/sdks/python/apache_beam/runners/direct/transform_evaluator.py
@@ -35,8 +35,10 @@ from apache_beam.transforms import sideinputs
 from apache_beam.transforms.window import GlobalWindows
 from apache_beam.transforms.window import WindowedValue
 from apache_beam.typehints.typecheck import OutputCheckWrapperDoFn
+from apache_beam.typehints.typecheck import OutputCheckWrapperNewDoFn
 from apache_beam.typehints.typecheck import TypeCheckError
 from apache_beam.typehints.typecheck import TypeCheckWrapperDoFn
+from apache_beam.typehints.typecheck import TypeCheckWrapperNewDoFn
 from apache_beam.utils import counters
 from apache_beam.utils.pipeline_options import TypeOptions
 
@@ -344,9 +346,18 @@ class _ParDoEvaluator(_TransformEvaluator):
     pipeline_options = self._evaluation_context.pipeline_options
     if (pipeline_options is not None
         and pipeline_options.view_as(TypeOptions).runtime_type_check):
-      dofn = TypeCheckWrapperDoFn(dofn, transform.get_type_hints())
+      # TODO(sourabhbajaj): Remove this if-else
+      if isinstance(dofn, core.NewDoFn):
+        dofn = TypeCheckWrapperNewDoFn(dofn, transform.get_type_hints())
+      else:
+        dofn = TypeCheckWrapperDoFn(dofn, transform.get_type_hints())
 
-    dofn = OutputCheckWrapperDoFn(dofn, self._applied_ptransform.full_label)
+    # TODO(sourabhbajaj): Remove this if-else
+    if isinstance(dofn, core.NewDoFn):
+      dofn = OutputCheckWrapperNewDoFn(
+          dofn, self._applied_ptransform.full_label)
+    else:
+      dofn = OutputCheckWrapperDoFn(dofn, self._applied_ptransform.full_label)
     self.runner = DoFnRunner(dofn, transform.args, transform.kwargs,
                              self._side_inputs,
                              self._applied_ptransform.inputs[0].windowing,

http://git-wip-us.apache.org/repos/asf/beam/blob/9e272ecf/sdks/python/apache_beam/transforms/core.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/transforms/core.py b/sdks/python/apache_beam/transforms/core.py
index 72f7cd4..70a03ae 100644
--- a/sdks/python/apache_beam/transforms/core.py
+++ b/sdks/python/apache_beam/transforms/core.py
@@ -109,6 +109,7 @@ class DoFnProcessContext(DoFnContext):
       self.timestamp = windowed_value.timestamp
       self.windows = windowed_value.windows
 
+  # TODO(sourabhbajaj): Move as we're trying to deprecate the use of context
   def aggregate_to(self, aggregator, input_value):
     """Provide a new input value for the aggregator.
 
@@ -119,6 +120,112 @@ class DoFnProcessContext(DoFnContext):
     self.state.counter_for(aggregator).update(input_value)
 
 
+class NewDoFn(WithTypeHints, HasDisplayData):
+  """A function object used by a transform with custom processing.
+
+  The ParDo transform is such a transform. The ParDo.apply
+  method will take an object of type DoFn and apply it to all elements of a
+  PCollection object.
+
+  In order to have concrete DoFn objects one has to subclass from DoFn and
+  define the desired behavior (start_bundle/finish_bundle and process) or wrap a
+  callable object using the CallableWrapperDoFn class.
+  """
+
+  ElementParam = 'ElementParam'
+  ContextParam = 'ContextParam'
+  SideInputParam = 'SideInputParam'
+  TimestampParam = 'TimestampParam'
+  WindowParam = 'WindowParam'
+
+  @staticmethod
+  def from_callable(fn):
+    return CallableWrapperDoFn(fn)
+
+  def default_label(self):
+    return self.__class__.__name__
+
+  def process(self, element, *args, **kwargs):
+    """Called for each element of a pipeline. The default arguments are needed
+    for the DoFnRunner to be able to pass the parameters correctly.
+
+    Args:
+      element: The element to be processed
+      context: a DoFnProcessContext object containing. See the
+        DoFnProcessContext documentation for details.
+      *args: side inputs
+      **kwargs: keyword side inputs
+    """
+    raise NotImplementedError
+
+  def start_bundle(self):
+    """Called before a bundle of elements is processed on a worker.
+
+    Elements to be processed are split into bundles and distributed
+    to workers. Before a worker calls process() on the first element
+    of its bundle, it calls this method.
+    """
+    pass
+
+  def finish_bundle(self):
+    """Called after a bundle of elements is processed on a worker.
+    """
+    pass
+
+  def get_function_arguments(self, func):
+    """Return the function arguments based on the name provided. If they have
+    a _inspect_function attached to the class then use that otherwise default
+    to the python inspect library.
+    """
+    func_name = '_inspect_%s' % func
+    if hasattr(self, func_name):
+      f = getattr(self, func_name)
+      return f()
+    else:
+      f = getattr(self, func)
+      return inspect.getargspec(f)
+
+  # TODO(sourabhbajaj): Do we want to remove the responsiblity of these from
+  # the DoFn or maybe the runner
+  def infer_output_type(self, input_type):
+    # TODO(robertwb): Side inputs types.
+    # TODO(robertwb): Assert compatibility with input type hint?
+    return self._strip_output_annotations(
+        trivial_inference.infer_return_type(self.process, [input_type]))
+
+  def _strip_output_annotations(self, type_hint):
+    annotations = (window.TimestampedValue, window.WindowedValue,
+                   pvalue.SideOutputValue)
+    # TODO(robertwb): These should be parameterized types that the
+    # type inferencer understands.
+    if (type_hint in annotations
+        or trivial_inference.element_type(type_hint) in annotations):
+      return Any
+    else:
+      return type_hint
+
+  def process_argspec_fn(self):
+    """Returns the Python callable that will eventually be invoked.
+
+    This should ideally be the user-level function that is called with
+    the main and (if any) side inputs, and is used to relate the type
+    hint parameters with the input parameters (e.g., by argument name).
+    """
+    return self.process
+
+  def is_process_bounded(self):
+    """Checks if an object is a bound method on an instance."""
+    if not isinstance(self.process, types.MethodType):
+      return False # Not a method
+    if self.process.im_self is None:
+      return False # Method is not bound
+    if issubclass(self.process.im_class, type) or \
+        self.process.im_class is types.ClassType:
+      return False # Method is a classmethod
+    return True
+
+
+# TODO(Sourabh): Remove after migration to NewDoFn
 class DoFn(WithTypeHints, HasDisplayData):
   """A function object used by a transform with custom processing.
 
@@ -577,7 +684,7 @@ class ParDo(PTransformWithSideInputs):
   def __init__(self, fn_or_label, *args, **kwargs):
     super(ParDo, self).__init__(fn_or_label, *args, **kwargs)
 
-    if not isinstance(self.fn, DoFn):
+    if not isinstance(self.fn, (DoFn, NewDoFn)):
       raise TypeError('ParDo must be called with a DoFn instance.')
 
   def default_type_hints(self):
@@ -588,7 +695,9 @@ class ParDo(PTransformWithSideInputs):
         self.fn.infer_output_type(input_type))
 
   def make_fn(self, fn):
-    return fn if isinstance(fn, DoFn) else CallableWrapperDoFn(fn)
+    if isinstance(fn, (DoFn, NewDoFn)):
+      return fn
+    return CallableWrapperDoFn(fn)
 
   def process_argspec_fn(self):
     return self.fn.process_argspec_fn()

http://git-wip-us.apache.org/repos/asf/beam/blob/9e272ecf/sdks/python/apache_beam/typehints/decorators.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/typehints/decorators.py b/sdks/python/apache_beam/typehints/decorators.py
index a300a3b..df15f1b 100644
--- a/sdks/python/apache_beam/typehints/decorators.py
+++ b/sdks/python/apache_beam/typehints/decorators.py
@@ -263,7 +263,7 @@ def getcallargs_forhints(func, *typeargs, **typekwargs):
     for k, var in enumerate(reversed(argspec.args)):
       if k >= len(argspec.defaults):
         break
-      if callargs.get(var, None) is argspec.defaults[-k]:
+      if callargs.get(var, None) is argspec.defaults[-k-1]:
         callargs[var] = typehints.Any
   # Patch up varargs and keywords
   if argspec.varargs:

http://git-wip-us.apache.org/repos/asf/beam/blob/9e272ecf/sdks/python/apache_beam/typehints/typecheck.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/typehints/typecheck.py b/sdks/python/apache_beam/typehints/typecheck.py
index d759d55..7a10a5a 100644
--- a/sdks/python/apache_beam/typehints/typecheck.py
+++ b/sdks/python/apache_beam/typehints/typecheck.py
@@ -24,6 +24,7 @@ import types
 
 from apache_beam.pvalue import SideOutputValue
 from apache_beam.transforms.core import DoFn
+from apache_beam.transforms.core import NewDoFn
 from apache_beam.transforms.window import WindowedValue
 from apache_beam.typehints import check_constraint
 from apache_beam.typehints import CompositeTypeHintError
@@ -162,3 +163,147 @@ class OutputCheckWrapperDoFn(DoFn):
                            'iterable. %s was returned instead.'
                            % type(output))
     return output
+
+
+class AbstractDoFnWrapper(NewDoFn):
+  """An abstract class to create wrapper around NewDoFn"""
+
+  def __init__(self, dofn):
+    super(AbstractDoFnWrapper, self).__init__()
+    self.dofn = dofn
+
+  def _inspect_start_bundle(self):
+    return self.dofn.get_function_arguments('start_bundle')
+
+  def _inspect_process(self):
+    return self.dofn.get_function_arguments('process')
+
+  def _inspect_finish_bundle(self):
+    return self.dofn.get_function_arguments('finish_bundle')
+
+  def wrapper(self, method, args, kwargs):
+    return method(*args, **kwargs)
+
+  def start_bundle(self, *args, **kwargs):
+    return self.wrapper(self.dofn.start_bundle, args, kwargs)
+
+  def process(self, *args, **kwargs):
+    return self.wrapper(self.dofn.process, args, kwargs)
+
+  def finish_bundle(self, *args, **kwargs):
+    return self.wrapper(self.dofn.finish_bundle, args, kwargs)
+
+  def is_process_bounded(self):
+    return self.dofn.is_process_bounded()
+
+
+class OutputCheckWrapperNewDoFn(AbstractDoFnWrapper):
+  """A DoFn that verifies against common errors in the output type."""
+
+  def __init__(self, dofn, full_label):
+    super(OutputCheckWrapperNewDoFn, self).__init__(dofn)
+    self.full_label = full_label
+
+  def wrapper(self, method, args, kwargs):
+    try:
+      result = method(*args, **kwargs)
+    except TypeCheckError as e:
+      error_msg = ('Runtime type violation detected within ParDo(%s): '
+                   '%s' % (self.full_label, e))
+      raise TypeCheckError, error_msg, sys.exc_info()[2]
+    else:
+      return self._check_type(result)
+
+  def _check_type(self, output):
+    if output is None:
+      return output
+    elif isinstance(output, (dict, basestring)):
+      object_type = type(output).__name__
+      raise TypeCheckError('Returning a %s from a ParDo or FlatMap is '
+                           'discouraged. Please use list("%s") if you really '
+                           'want this behavior.' %
+                           (object_type, output))
+    elif not isinstance(output, collections.Iterable):
+      raise TypeCheckError('FlatMap and ParDo must return an '
+                           'iterable. %s was returned instead.'
+                           % type(output))
+    return output
+
+
+class TypeCheckWrapperNewDoFn(AbstractDoFnWrapper):
+  """A wrapper around a DoFn which performs type-checking of input and output.
+  """
+
+  def __init__(self, dofn, type_hints, label=None):
+    super(TypeCheckWrapperNewDoFn, self).__init__(dofn)
+    self.dofn = dofn
+    self._process_fn = self.dofn.process_argspec_fn()
+    if type_hints.input_types:
+      input_args, input_kwargs = type_hints.input_types
+      self._input_hints = getcallargs_forhints(
+          self._process_fn, *input_args, **input_kwargs)
+    else:
+      self._input_hints = None
+    # TODO(robertwb): Multi-output.
+    self._output_type_hint = type_hints.simple_output_type(label)
+
+  def wrapper(self, method, args, kwargs):
+    result = method(*args, **kwargs)
+    return self._type_check_result(result)
+
+  def process(self, *args, **kwargs):
+    if self._input_hints:
+      actual_inputs = inspect.getcallargs(self._process_fn, *args, **kwargs)
+      for var, hint in self._input_hints.items():
+        if hint is actual_inputs[var]:
+          # self parameter
+          continue
+        _check_instance_type(hint, actual_inputs[var], var, True)
+    return self._type_check_result(self.dofn.process(*args, **kwargs))
+
+  def _type_check_result(self, transform_results):
+    if self._output_type_hint is None or transform_results is None:
+      return transform_results
+
+    def type_check_output(o):
+      # TODO(robertwb): Multi-output.
+      x = o.value if isinstance(o, (SideOutputValue, WindowedValue)) else o
+      self._type_check(self._output_type_hint, x, is_input=False)
+
+    # If the return type is a generator, then we will need to interleave our
+    # type-checking with its normal iteration so we don't deplete the
+    # generator initially just by type-checking its yielded contents.
+    if isinstance(transform_results, types.GeneratorType):
+      return GeneratorWrapper(transform_results, type_check_output)
+    else:
+      for o in transform_results:
+        type_check_output(o)
+      return transform_results
+
+  def _type_check(self, type_constraint, datum, is_input):
+    """Typecheck a PTransform related datum according to a type constraint.
+
+    This function is used to optionally type-check either an input or an output
+    to a PTransform.
+
+    Args:
+        type_constraint: An instance of a typehints.TypeContraint, one of the
+          white-listed builtin Python types, or a custom user class.
+        datum: An instance of a Python object.
+        is_input: True if 'datum' is an input to a PTransform's DoFn. False
+          otherwise.
+
+    Raises:
+      TypeError: If 'datum' fails to type-check according to 'type_constraint'.
+    """
+    datum_type = 'input' if is_input else 'output'
+
+    try:
+      check_constraint(type_constraint, datum)
+    except CompositeTypeHintError as e:
+      raise TypeCheckError, e.message, sys.exc_info()[2]
+    except SimpleTypeHintError:
+      error_msg = ("According to type-hint expected %s should be of type %s. "
+                   "Instead, received '%s', an instance of type %s."
+                   % (datum_type, type_constraint, datum, type(datum)))
+      raise TypeCheckError, error_msg, sys.exc_info()[2]


[40/50] [abbrv] beam git commit: [BEAM-843] Use New DoFn Directly in Flink Runner

Posted by da...@apache.org.
[BEAM-843] Use New DoFn Directly in Flink Runner


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

Branch: refs/heads/master
Commit: 4aaaf8fb94222e6e283fa79d9c7dc8d9a730d278
Parents: 27cf68e
Author: JingsongLi <lz...@aliyun.com>
Authored: Wed Jan 18 11:34:06 2017 +0800
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jan 30 12:38:38 2017 -0800

----------------------------------------------------------------------
 .../GroupAlsoByWindowViaWindowSetNewDoFn.java   | 156 +++++++++++++++++++
 .../wrappers/streaming/DoFnOperator.java        |  69 ++++----
 .../wrappers/streaming/WindowDoFnOperator.java  | 143 +++++++++--------
 3 files changed, 264 insertions(+), 104 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/4aaaf8fb/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetNewDoFn.java
----------------------------------------------------------------------
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetNewDoFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetNewDoFn.java
new file mode 100644
index 0000000..cff6e00
--- /dev/null
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetNewDoFn.java
@@ -0,0 +1,156 @@
+/*
+ * 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.
+ */
+package org.apache.beam.runners.core;
+
+import java.util.Collection;
+import org.apache.beam.runners.core.triggers.ExecutableTriggerStateMachine;
+import org.apache.beam.runners.core.triggers.TriggerStateMachines;
+import org.apache.beam.sdk.transforms.Aggregator;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.Sum;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.util.SideInputReader;
+import org.apache.beam.sdk.util.SystemDoFnInternal;
+import org.apache.beam.sdk.util.TimerInternals;
+import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.util.WindowingStrategy;
+import org.apache.beam.sdk.util.state.StateInternals;
+import org.apache.beam.sdk.util.state.StateInternalsFactory;
+import org.apache.beam.sdk.util.state.TimerInternalsFactory;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.TupleTag;
+import org.joda.time.Instant;
+
+/**
+ * A general {@link GroupAlsoByWindowsDoFn}. This delegates all of the logic to the
+ * {@link ReduceFnRunner}.
+ */
+@SystemDoFnInternal
+public class GroupAlsoByWindowViaWindowSetNewDoFn<
+        K, InputT, OutputT, W extends BoundedWindow, RinT extends KeyedWorkItem<K, InputT>>
+    extends DoFn<RinT, KV<K, OutputT>> {
+
+  private static final long serialVersionUID = 1L;
+
+  public static <K, InputT, OutputT, W extends BoundedWindow>
+      DoFn<KeyedWorkItem<K, InputT>, KV<K, OutputT>> create(
+          WindowingStrategy<?, W> strategy,
+          StateInternalsFactory<K> stateInternalsFactory,
+          TimerInternalsFactory<K> timerInternalsFactory,
+          SideInputReader sideInputReader,
+          SystemReduceFn<K, InputT, ?, OutputT, W> reduceFn,
+          DoFnRunners.OutputManager outputManager,
+          TupleTag<KV<K, OutputT>> mainTag) {
+    return new GroupAlsoByWindowViaWindowSetNewDoFn<>(
+            strategy, stateInternalsFactory, timerInternalsFactory, sideInputReader,
+            reduceFn, outputManager, mainTag);
+  }
+
+  protected final Aggregator<Long, Long> droppedDueToClosedWindow =
+      createAggregator(
+          GroupAlsoByWindowsDoFn.DROPPED_DUE_TO_CLOSED_WINDOW_COUNTER, Sum.ofLongs());
+  protected final Aggregator<Long, Long> droppedDueToLateness =
+      createAggregator(GroupAlsoByWindowsDoFn.DROPPED_DUE_TO_LATENESS_COUNTER, Sum.ofLongs());
+  private final WindowingStrategy<Object, W> windowingStrategy;
+  private SystemReduceFn<K, InputT, ?, OutputT, W> reduceFn;
+  private transient StateInternalsFactory<K> stateInternalsFactory;
+  private transient TimerInternalsFactory<K> timerInternalsFactory;
+  private transient SideInputReader sideInputReader;
+  private transient DoFnRunners.OutputManager outputManager;
+  private TupleTag<KV<K, OutputT>> mainTag;
+
+  public GroupAlsoByWindowViaWindowSetNewDoFn(
+          WindowingStrategy<?, W> windowingStrategy,
+          StateInternalsFactory<K> stateInternalsFactory,
+          TimerInternalsFactory<K> timerInternalsFactory,
+          SideInputReader sideInputReader,
+          SystemReduceFn<K, InputT, ?, OutputT, W> reduceFn,
+          DoFnRunners.OutputManager outputManager,
+          TupleTag<KV<K, OutputT>> mainTag) {
+    this.timerInternalsFactory = timerInternalsFactory;
+    this.sideInputReader = sideInputReader;
+    this.outputManager = outputManager;
+    this.mainTag = mainTag;
+    @SuppressWarnings("unchecked")
+    WindowingStrategy<Object, W> noWildcard = (WindowingStrategy<Object, W>) windowingStrategy;
+    this.windowingStrategy = noWildcard;
+    this.reduceFn = reduceFn;
+    this.stateInternalsFactory = stateInternalsFactory;
+  }
+
+  private OutputWindowedValue<KV<K, OutputT>> outputWindowedValue() {
+    return new OutputWindowedValue<KV<K, OutputT>>() {
+      @Override
+      public void outputWindowedValue(
+              KV<K, OutputT> output,
+              Instant timestamp,
+              Collection<? extends BoundedWindow> windows,
+              PaneInfo pane) {
+        outputManager.output(mainTag,
+                WindowedValue.of(output, timestamp, windows, pane));
+      }
+
+      @Override
+      public <SideOutputT> void sideOutputWindowedValue(
+              TupleTag<SideOutputT> tag,
+              SideOutputT output,
+              Instant timestamp,
+              Collection<? extends BoundedWindow> windows,
+              PaneInfo pane) {
+        outputManager.output(tag,
+                WindowedValue.of(output, timestamp, windows, pane));
+      }
+    };
+  }
+
+  @ProcessElement
+  public void processElement(ProcessContext c) throws Exception {
+    KeyedWorkItem<K, InputT> keyedWorkItem = c.element();
+
+    K key = keyedWorkItem.key();
+    StateInternals<K> stateInternals = stateInternalsFactory.stateInternalsForKey(key);
+    TimerInternals timerInternals = timerInternalsFactory.timerInternalsForKey(key);
+
+    ReduceFnRunner<K, InputT, OutputT, W> reduceFnRunner =
+        new ReduceFnRunner<>(
+            key,
+            windowingStrategy,
+            ExecutableTriggerStateMachine.create(
+                TriggerStateMachines.stateMachineForTrigger(windowingStrategy.getTrigger())),
+            stateInternals,
+            timerInternals,
+            outputWindowedValue(),
+            sideInputReader,
+            droppedDueToClosedWindow,
+            reduceFn,
+            c.getPipelineOptions());
+
+    reduceFnRunner.processElements(keyedWorkItem.elementsIterable());
+    reduceFnRunner.onTimers(keyedWorkItem.timersIterable());
+    reduceFnRunner.persist();
+  }
+
+  public OldDoFn<KeyedWorkItem<K, InputT>, KV<K, OutputT>> asDoFn() {
+    throw new RuntimeException("Not implement!");
+  }
+
+  public Aggregator<Long, Long> getDroppedDueToLatenessAggregator() {
+    return droppedDueToLateness;
+  }
+}

http://git-wip-us.apache.org/repos/asf/beam/blob/4aaaf8fb/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
index ac85b3c..de0264a 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
@@ -28,12 +28,11 @@ import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+
 import org.apache.beam.runners.core.AggregatorFactory;
-import org.apache.beam.runners.core.DoFnAdapters;
 import org.apache.beam.runners.core.DoFnRunner;
 import org.apache.beam.runners.core.DoFnRunners;
 import org.apache.beam.runners.core.ExecutionContext;
-import org.apache.beam.runners.core.OldDoFn;
 import org.apache.beam.runners.core.PushbackSideInputDoFnRunner;
 import org.apache.beam.runners.core.SideInputHandler;
 import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions;
@@ -45,6 +44,8 @@ import org.apache.beam.sdk.transforms.Aggregator;
 import org.apache.beam.sdk.transforms.Combine;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.join.RawUnionValue;
+import org.apache.beam.sdk.transforms.reflect.DoFnInvoker;
+import org.apache.beam.sdk.transforms.reflect.DoFnInvokers;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.util.CoderUtils;
 import org.apache.beam.sdk.util.NullSideInputReader;
@@ -78,10 +79,10 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.runtime.tasks.StreamTaskState;
 
 /**
- * Flink operator for executing {@link OldDoFn DoFns}.
+ * Flink operator for executing {@link DoFn DoFns}.
  *
- * @param <InputT> the input type of the {@link OldDoFn}
- * @param <FnOutputT> the output type of the {@link OldDoFn}
+ * @param <InputT> the input type of the {@link DoFn}
+ * @param <FnOutputT> the output type of the {@link DoFn}
  * @param <OutputT> the output type of the operator, this can be different from the fn output
  *                 type when we have side outputs
  */
@@ -90,7 +91,7 @@ public class DoFnOperator<InputT, FnOutputT, OutputT>
     implements OneInputStreamOperator<WindowedValue<InputT>, OutputT>,
       TwoInputStreamOperator<WindowedValue<InputT>, RawUnionValue, OutputT> {
 
-  protected OldDoFn<InputT, FnOutputT> oldDoFn;
+  protected DoFn<InputT, FnOutputT> doFn;
 
   protected final SerializedPipelineOptions serializedOptions;
 
@@ -108,6 +109,12 @@ public class DoFnOperator<InputT, FnOutputT, OutputT>
 
   protected transient SideInputHandler sideInputHandler;
 
+  protected transient SideInputReader sideInputReader;
+
+  protected transient DoFnRunners.OutputManager outputManager;
+
+  private transient DoFnInvoker<InputT, FnOutputT> doFnInvoker;
+
   protected transient long currentInputWatermark;
 
   protected transient long currentOutputWatermark;
@@ -120,9 +127,8 @@ public class DoFnOperator<InputT, FnOutputT, OutputT>
 
   private transient Map<String, KvStateSnapshot<?, ?, ?, ?, ?>> restoredSideInputState;
 
-  @Deprecated
   public DoFnOperator(
-      OldDoFn<InputT, FnOutputT> oldDoFn,
+      DoFn<InputT, FnOutputT> doFn,
       TypeInformation<WindowedValue<InputT>> inputType,
       TupleTag<FnOutputT> mainOutputTag,
       List<TupleTag<?>> sideOutputTags,
@@ -131,7 +137,7 @@ public class DoFnOperator<InputT, FnOutputT, OutputT>
       Map<Integer, PCollectionView<?>> sideInputTagMapping,
       Collection<PCollectionView<?>> sideInputs,
       PipelineOptions options) {
-    this.oldDoFn = oldDoFn;
+    this.doFn = doFn;
     this.mainOutputTag = mainOutputTag;
     this.sideOutputTags = sideOutputTags;
     this.sideInputTagMapping = sideInputTagMapping;
@@ -152,44 +158,20 @@ public class DoFnOperator<InputT, FnOutputT, OutputT>
     setChainingStrategy(ChainingStrategy.ALWAYS);
   }
 
-  public DoFnOperator(
-      DoFn<InputT, FnOutputT> doFn,
-      TypeInformation<WindowedValue<InputT>> inputType,
-      TupleTag<FnOutputT> mainOutputTag,
-      List<TupleTag<?>> sideOutputTags,
-      OutputManagerFactory<OutputT> outputManagerFactory,
-      WindowingStrategy<?, ?> windowingStrategy,
-      Map<Integer, PCollectionView<?>> sideInputTagMapping,
-      Collection<PCollectionView<?>> sideInputs,
-      PipelineOptions options) {
-    this(
-        DoFnAdapters.toOldDoFn(doFn),
-        inputType,
-        mainOutputTag,
-        sideOutputTags,
-        outputManagerFactory,
-        windowingStrategy,
-        sideInputTagMapping,
-        sideInputs,
-        options);
-  }
-
   protected ExecutionContext.StepContext createStepContext() {
     return new StepContext();
   }
 
   // allow overriding this in WindowDoFnOperator because this one dynamically creates
   // the DoFn
-  protected OldDoFn<InputT, FnOutputT> getOldDoFn() {
-    return oldDoFn;
+  protected DoFn<InputT, FnOutputT> getDoFn() {
+    return doFn;
   }
 
   @Override
   public void open() throws Exception {
     super.open();
 
-    this.oldDoFn = getOldDoFn();
-
     currentInputWatermark = Long.MIN_VALUE;
     currentOutputWatermark = currentInputWatermark;
 
@@ -214,7 +196,8 @@ public class DoFnOperator<InputT, FnOutputT, OutputT>
       }
     };
 
-    SideInputReader sideInputReader = NullSideInputReader.of(sideInputs);
+    sideInputReader = NullSideInputReader.of(sideInputs);
+
     if (!sideInputs.isEmpty()) {
       String operatorIdentifier =
           this.getClass().getSimpleName() + "_"
@@ -244,11 +227,18 @@ public class DoFnOperator<InputT, FnOutputT, OutputT>
       sideInputReader = sideInputHandler;
     }
 
+    outputManager = outputManagerFactory.create(output);
+
+    this.doFn = getDoFn();
+    doFnInvoker = DoFnInvokers.invokerFor(doFn);
+
+    doFnInvoker.invokeSetup();
+
     DoFnRunner<InputT, FnOutputT> doFnRunner = DoFnRunners.simpleRunner(
         serializedOptions.getPipelineOptions(),
-        oldDoFn,
+        doFn,
         sideInputReader,
-        outputManagerFactory.create(output),
+        outputManager,
         mainOutputTag,
         sideOutputTags,
         createStepContext(),
@@ -258,13 +248,12 @@ public class DoFnOperator<InputT, FnOutputT, OutputT>
     pushbackDoFnRunner =
         PushbackSideInputDoFnRunner.create(doFnRunner, sideInputs, sideInputHandler);
 
-    oldDoFn.setup();
   }
 
   @Override
   public void close() throws Exception {
     super.close();
-    oldDoFn.teardown();
+    doFnInvoker.invokeTeardown();
   }
 
   protected final long getPushbackWatermarkHold() {

http://git-wip-us.apache.org/repos/asf/beam/blob/4aaaf8fb/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java
----------------------------------------------------------------------
diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java
index d4273b2..74614ad 100644
--- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java
+++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java
@@ -38,11 +38,11 @@ import java.util.Queue;
 import java.util.Set;
 import java.util.concurrent.ScheduledFuture;
 import javax.annotation.Nullable;
+
 import org.apache.beam.runners.core.ExecutionContext;
-import org.apache.beam.runners.core.GroupAlsoByWindowViaWindowSetDoFn;
+import org.apache.beam.runners.core.GroupAlsoByWindowViaWindowSetNewDoFn;
 import org.apache.beam.runners.core.KeyedWorkItem;
 import org.apache.beam.runners.core.KeyedWorkItems;
-import org.apache.beam.runners.core.OldDoFn;
 import org.apache.beam.runners.core.SystemReduceFn;
 import org.apache.beam.runners.flink.translation.wrappers.DataInputViewWrapper;
 import org.apache.beam.sdk.coders.Coder;
@@ -56,6 +56,7 @@ import org.apache.beam.sdk.util.WindowingStrategy;
 import org.apache.beam.sdk.util.state.StateInternals;
 import org.apache.beam.sdk.util.state.StateInternalsFactory;
 import org.apache.beam.sdk.util.state.StateNamespace;
+import org.apache.beam.sdk.util.state.TimerInternalsFactory;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.TupleTag;
@@ -91,6 +92,7 @@ public class WindowDoFnOperator<K, InputT, OutputT>
   private transient Map<Long, ScheduledFuture<?>> processingTimeTimerFutures;
 
   private transient FlinkStateInternals<K> stateInternals;
+  private transient FlinkTimerInternals timerInternals;
 
   private final SystemReduceFn<K, InputT, ?, OutputT, BoundedWindow> systemReduceFn;
 
@@ -106,7 +108,7 @@ public class WindowDoFnOperator<K, InputT, OutputT>
       PipelineOptions options,
       Coder<K> keyCoder) {
     super(
-        (OldDoFn<KeyedWorkItem<K, InputT>, KV<K, OutputT>>) null,
+        null,
         inputType,
         mainOutputTag,
         sideOutputTags,
@@ -124,7 +126,7 @@ public class WindowDoFnOperator<K, InputT, OutputT>
   }
 
   @Override
-  protected OldDoFn<KeyedWorkItem<K, InputT>, KV<K, OutputT>> getOldDoFn() {
+  protected DoFn<KeyedWorkItem<K, InputT>, KV<K, OutputT>> getDoFn() {
     StateInternalsFactory<K> stateInternalsFactory = new StateInternalsFactory<K>() {
       @Override
       public StateInternals<K> stateInternalsForKey(K key) {
@@ -133,15 +135,23 @@ public class WindowDoFnOperator<K, InputT, OutputT>
         return stateInternals;
       }
     };
+    TimerInternalsFactory<K> timerInternalsFactory = new TimerInternalsFactory<K>() {
+      @Override
+      public TimerInternals timerInternalsForKey(K key) {
+        //this will implicitly be keyed like the StateInternalsFactory
+        return timerInternals;
+      }
+    };
 
     // we have to do the unchecked cast because GroupAlsoByWindowViaWindowSetDoFn.create
     // has the window type as generic parameter while WindowingStrategy is almost always
     // untyped.
     @SuppressWarnings("unchecked")
-    OldDoFn<KeyedWorkItem<K, InputT>, KV<K, OutputT>> oldDoFn =
-        GroupAlsoByWindowViaWindowSetDoFn.create(
-            windowingStrategy, stateInternalsFactory, (SystemReduceFn) systemReduceFn);
-    return oldDoFn;
+    DoFn<KeyedWorkItem<K, InputT>, KV<K, OutputT>> doFn =
+        GroupAlsoByWindowViaWindowSetNewDoFn.create(
+            windowingStrategy, stateInternalsFactory, timerInternalsFactory, sideInputReader,
+                (SystemReduceFn) systemReduceFn, outputManager, mainOutputTag);
+    return doFn;
   }
 
 
@@ -183,6 +193,7 @@ public class WindowDoFnOperator<K, InputT, OutputT>
     processingTimeTimerFutures = new HashMap<>();
 
     stateInternals = new FlinkStateInternals<>(getStateBackend(), keyCoder);
+    timerInternals = new FlinkTimerInternals();
 
     // call super at the end because this will call getDoFn() which requires stateInternals
     // to be set
@@ -448,75 +459,79 @@ public class WindowDoFnOperator<K, InputT, OutputT>
 
     @Override
     public TimerInternals timerInternals() {
-      return new TimerInternals() {
-        @Override
-        public void setTimer(
+      return timerInternals;
+    }
+  }
+
+  private class FlinkTimerInternals implements TimerInternals {
+
+    @Override
+    public void setTimer(
             StateNamespace namespace, String timerId, Instant target, TimeDomain timeDomain) {
-          throw new UnsupportedOperationException("Setting a timer by ID is not yet supported.");
-        }
+      throw new UnsupportedOperationException("Setting a timer by ID is not yet supported.");
+    }
 
-        @Deprecated
-        @Override
-        public void setTimer(TimerData timerKey) {
-          if (timerKey.getDomain().equals(TimeDomain.EVENT_TIME)) {
-            registerEventTimeTimer(timerKey);
-          } else if (timerKey.getDomain().equals(TimeDomain.PROCESSING_TIME)) {
-            registerProcessingTimeTimer(timerKey);
-          } else {
-            throw new UnsupportedOperationException(
+    @Deprecated
+    @Override
+    public void setTimer(TimerData timerKey) {
+      if (timerKey.getDomain().equals(TimeDomain.EVENT_TIME)) {
+        registerEventTimeTimer(timerKey);
+      } else if (timerKey.getDomain().equals(TimeDomain.PROCESSING_TIME)) {
+        registerProcessingTimeTimer(timerKey);
+      } else {
+        throw new UnsupportedOperationException(
                 "Unsupported time domain: " + timerKey.getDomain());
-          }
-        }
+      }
+    }
 
-        @Override
-        public void deleteTimer(StateNamespace namespace, String timerId, TimeDomain timeDomain) {
-          throw new UnsupportedOperationException(
+    @Deprecated
+    @Override
+    public void deleteTimer(StateNamespace namespace, String timerId) {
+      throw new UnsupportedOperationException(
               "Canceling of a timer by ID is not yet supported.");
-        }
+    }
 
-        @Deprecated
-        @Override
-        public void deleteTimer(StateNamespace namespace, String timerId) {
-          throw new UnsupportedOperationException(
-              "Canceling of a timer by ID is not yet supported.");
-        }
+    @Override
+    public void deleteTimer(StateNamespace namespace, String timerId, TimeDomain timeDomain) {
+      throw new UnsupportedOperationException(
+          "Canceling of a timer by ID is not yet supported.");
+    }
 
-        @Deprecated
-        @Override
-        public void deleteTimer(TimerData timerKey) {
-          if (timerKey.getDomain().equals(TimeDomain.EVENT_TIME)) {
-            deleteEventTimeTimer(timerKey);
-          } else if (timerKey.getDomain().equals(TimeDomain.PROCESSING_TIME)) {
-            deleteProcessingTimeTimer(timerKey);
-          } else {
-            throw new UnsupportedOperationException(
+    @Deprecated
+    @Override
+    public void deleteTimer(TimerData timerKey) {
+      if (timerKey.getDomain().equals(TimeDomain.EVENT_TIME)) {
+        deleteEventTimeTimer(timerKey);
+      } else if (timerKey.getDomain().equals(TimeDomain.PROCESSING_TIME)) {
+        deleteProcessingTimeTimer(timerKey);
+      } else {
+        throw new UnsupportedOperationException(
                 "Unsupported time domain: " + timerKey.getDomain());
-          }
-        }
+      }
+    }
 
-        @Override
-        public Instant currentProcessingTime() {
-          return new Instant(getCurrentProcessingTime());
-        }
+    @Override
+    public Instant currentProcessingTime() {
+      return new Instant(getCurrentProcessingTime());
+    }
 
-        @Nullable
-        @Override
-        public Instant currentSynchronizedProcessingTime() {
-          return new Instant(getCurrentProcessingTime());
-        }
+    @Nullable
+    @Override
+    public Instant currentSynchronizedProcessingTime() {
+      return new Instant(getCurrentProcessingTime());
+    }
 
-        @Override
-        public Instant currentInputWatermarkTime() {
-          return new Instant(Math.min(currentInputWatermark, getPushbackWatermarkHold()));
-        }
+    @Override
+    public Instant currentInputWatermarkTime() {
+      return new Instant(Math.min(currentInputWatermark, getPushbackWatermarkHold()));
+    }
 
-        @Nullable
-        @Override
-        public Instant currentOutputWatermarkTime() {
-          return new Instant(currentOutputWatermark);
-        }
-      };
+    @Nullable
+    @Override
+    public Instant currentOutputWatermarkTime() {
+      return new Instant(currentOutputWatermark);
     }
+
   }
 
 }


[50/50] [abbrv] beam git commit: This closes #1872

Posted by da...@apache.org.
This closes #1872


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

Branch: refs/heads/master
Commit: c3b97a2878a6ccb7b380cb7724ee0719a1d25d2e
Parents: 847e4e9 2d7ce32
Author: Davor Bonaci <da...@google.com>
Authored: Mon Jan 30 14:59:03 2017 -0800
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jan 30 14:59:03 2017 -0800

----------------------------------------------------------------------
 .gitignore                                      |   10 +
 .travis.yml                                     |   21 +-
 pom.xml                                         |   20 +
 sdks/pom.xml                                    |    2 +
 sdks/python/.pylintrc                           |  164 +
 sdks/python/MANIFEST.in                         |   19 +
 sdks/python/README.md                           |  372 ++
 sdks/python/apache_beam/__init__.py             |   82 +
 sdks/python/apache_beam/coders/__init__.py      |   19 +
 sdks/python/apache_beam/coders/coder_impl.pxd   |  143 +
 sdks/python/apache_beam/coders/coder_impl.py    |  597 +++
 sdks/python/apache_beam/coders/coders.py        |  707 +++
 sdks/python/apache_beam/coders/coders_test.py   |  115 +
 .../apache_beam/coders/coders_test_common.py    |  355 ++
 .../apache_beam/coders/fast_coders_test.py      |   37 +
 sdks/python/apache_beam/coders/observable.py    |   38 +
 .../apache_beam/coders/observable_test.py       |   57 +
 .../coders/proto2_coder_test_messages_pb2.py    |  318 ++
 .../apache_beam/coders/slow_coders_test.py      |   39 +
 sdks/python/apache_beam/coders/slow_stream.py   |  154 +
 .../apache_beam/coders/standard_coders.yaml     |   67 +
 .../apache_beam/coders/standard_coders_test.py  |  136 +
 sdks/python/apache_beam/coders/stream.pxd       |   63 +
 sdks/python/apache_beam/coders/stream.pyx       |  215 +
 sdks/python/apache_beam/coders/stream_test.py   |  169 +
 sdks/python/apache_beam/coders/typecoders.py    |  186 +
 .../apache_beam/coders/typecoders_test.py       |  117 +
 sdks/python/apache_beam/error.py                |   42 +
 sdks/python/apache_beam/examples/__init__.py    |   16 +
 .../apache_beam/examples/complete/__init__.py   |   16 +
 .../examples/complete/autocomplete.py           |   89 +
 .../examples/complete/autocomplete_test.py      |   52 +
 .../examples/complete/estimate_pi.py            |  125 +
 .../examples/complete/estimate_pi_test.py       |   52 +
 .../examples/complete/juliaset/__init__.py      |   16 +
 .../complete/juliaset/juliaset/__init__.py      |   16 +
 .../complete/juliaset/juliaset/juliaset.py      |  123 +
 .../complete/juliaset/juliaset/juliaset_test.py |   86 +
 .../examples/complete/juliaset/juliaset_main.py |   58 +
 .../examples/complete/juliaset/setup.py         |  116 +
 .../apache_beam/examples/complete/tfidf.py      |  208 +
 .../apache_beam/examples/complete/tfidf_test.py |   92 +
 .../examples/complete/top_wikipedia_sessions.py |  180 +
 .../complete/top_wikipedia_sessions_test.py     |   62 +
 .../apache_beam/examples/cookbook/__init__.py   |   16 +
 .../examples/cookbook/bigquery_schema.py        |  130 +
 .../examples/cookbook/bigquery_side_input.py    |  123 +
 .../cookbook/bigquery_side_input_test.py        |   54 +
 .../examples/cookbook/bigquery_tornadoes.py     |   99 +
 .../cookbook/bigquery_tornadoes_test.py         |   44 +
 .../apache_beam/examples/cookbook/bigshuffle.py |   95 +
 .../examples/cookbook/bigshuffle_test.py        |   63 +
 .../apache_beam/examples/cookbook/coders.py     |  101 +
 .../examples/cookbook/coders_test.py            |   49 +
 .../examples/cookbook/combiners_test.py         |   74 +
 .../examples/cookbook/custom_ptransform.py      |  134 +
 .../examples/cookbook/custom_ptransform_test.py |   53 +
 .../examples/cookbook/datastore_wordcount.py    |  256 ++
 .../apache_beam/examples/cookbook/filters.py    |  107 +
 .../examples/cookbook/filters_test.py           |   69 +
 .../examples/cookbook/group_with_coder.py       |  122 +
 .../examples/cookbook/group_with_coder_test.py  |   89 +
 .../examples/cookbook/mergecontacts.py          |  133 +
 .../examples/cookbook/mergecontacts_test.py     |  125 +
 .../examples/cookbook/multiple_output_pardo.py  |  181 +
 .../cookbook/multiple_output_pardo_test.py      |   72 +
 .../apache_beam/examples/snippets/__init__.py   |   16 +
 .../apache_beam/examples/snippets/snippets.py   | 1142 +++++
 .../examples/snippets/snippets_test.py          |  758 ++++
 .../apache_beam/examples/streaming_wordcap.py   |   64 +
 .../apache_beam/examples/streaming_wordcount.py |   74 +
 sdks/python/apache_beam/examples/wordcount.py   |  109 +
 .../apache_beam/examples/wordcount_debugging.py |  166 +
 .../examples/wordcount_debugging_test.py        |   59 +
 .../apache_beam/examples/wordcount_it_test.py   |   59 +
 .../apache_beam/examples/wordcount_minimal.py   |  121 +
 .../examples/wordcount_minimal_test.py          |   59 +
 .../apache_beam/examples/wordcount_test.py      |   58 +
 sdks/python/apache_beam/internal/__init__.py    |   16 +
 sdks/python/apache_beam/internal/apiclient.py   |  639 +++
 .../apache_beam/internal/apiclient_test.py      |   42 +
 sdks/python/apache_beam/internal/auth.py        |  185 +
 sdks/python/apache_beam/internal/auth_test.py   |   44 +
 .../apache_beam/internal/clients/__init__.py    |   16 +
 .../internal/clients/bigquery/__init__.py       |   27 +
 .../clients/bigquery/bigquery_v2_client.py      |  659 +++
 .../clients/bigquery/bigquery_v2_messages.py    | 1910 ++++++++
 .../internal/clients/dataflow/__init__.py       |   27 +
 .../clients/dataflow/dataflow_v1b3_client.py    |  683 +++
 .../clients/dataflow/dataflow_v1b3_messages.py  | 4173 ++++++++++++++++++
 .../clients/dataflow/message_matchers.py        |  124 +
 .../clients/dataflow/message_matchers_test.py   |   69 +
 .../internal/clients/storage/__init__.py        |   27 +
 .../clients/storage/storage_v1_client.py        | 1038 +++++
 .../clients/storage/storage_v1_messages.py      | 1920 ++++++++
 sdks/python/apache_beam/internal/json_value.py  |  141 +
 .../apache_beam/internal/json_value_test.py     |   84 +
 sdks/python/apache_beam/internal/module_test.py |   62 +
 sdks/python/apache_beam/internal/pickler.py     |  230 +
 .../python/apache_beam/internal/pickler_test.py |   84 +
 sdks/python/apache_beam/internal/util.py        |  127 +
 sdks/python/apache_beam/internal/util_test.py   |   61 +
 .../python/apache_beam/internal/windmill_pb2.py | 2278 ++++++++++
 .../internal/windmill_service_pb2.py            |  164 +
 sdks/python/apache_beam/io/__init__.py          |   31 +
 sdks/python/apache_beam/io/avroio.py            |  345 ++
 sdks/python/apache_beam/io/avroio_test.py       |  349 ++
 sdks/python/apache_beam/io/bigquery.py          | 1081 +++++
 sdks/python/apache_beam/io/bigquery_test.py     |  806 ++++
 sdks/python/apache_beam/io/concat_source.py     |  263 ++
 .../python/apache_beam/io/concat_source_test.py |  231 +
 .../python/apache_beam/io/datastore/__init__.py |   16 +
 .../apache_beam/io/datastore/v1/__init__.py     |   16 +
 .../apache_beam/io/datastore/v1/datastoreio.py  |  391 ++
 .../io/datastore/v1/datastoreio_test.py         |  247 ++
 .../io/datastore/v1/fake_datastore.py           |   92 +
 .../apache_beam/io/datastore/v1/helper.py       |  267 ++
 .../apache_beam/io/datastore/v1/helper_test.py  |  256 ++
 .../io/datastore/v1/query_splitter.py           |  269 ++
 .../io/datastore/v1/query_splitter_test.py      |  201 +
 sdks/python/apache_beam/io/filebasedsource.py   |  329 ++
 .../apache_beam/io/filebasedsource_test.py      |  708 +++
 sdks/python/apache_beam/io/fileio.py            |  797 ++++
 sdks/python/apache_beam/io/fileio_test.py       |  289 ++
 sdks/python/apache_beam/io/gcsio.py             |  820 ++++
 sdks/python/apache_beam/io/gcsio_test.py        |  748 ++++
 sdks/python/apache_beam/io/iobase.py            |  854 ++++
 sdks/python/apache_beam/io/pubsub.py            |   91 +
 sdks/python/apache_beam/io/pubsub_test.py       |   62 +
 sdks/python/apache_beam/io/range_trackers.py    |  493 +++
 .../apache_beam/io/range_trackers_test.py       |  538 +++
 sdks/python/apache_beam/io/source_test_utils.py |  641 +++
 .../apache_beam/io/source_test_utils_test.py    |  123 +
 sdks/python/apache_beam/io/sources_test.py      |  111 +
 sdks/python/apache_beam/io/textio.py            |  318 ++
 sdks/python/apache_beam/io/textio_test.py       |  537 +++
 sdks/python/apache_beam/io/tfrecordio.py        |  271 ++
 sdks/python/apache_beam/io/tfrecordio_test.py   |  389 ++
 sdks/python/apache_beam/metrics/__init__.py     |   17 +
 sdks/python/apache_beam/metrics/cells.py        |  315 ++
 sdks/python/apache_beam/metrics/cells_test.py   |  143 +
 sdks/python/apache_beam/metrics/execution.pxd   |   31 +
 sdks/python/apache_beam/metrics/execution.py    |  228 +
 .../apache_beam/metrics/execution_test.py       |  131 +
 sdks/python/apache_beam/metrics/metric.py       |  165 +
 sdks/python/apache_beam/metrics/metric_test.py  |   85 +
 sdks/python/apache_beam/metrics/metricbase.py   |   81 +
 sdks/python/apache_beam/pipeline.py             |  442 ++
 sdks/python/apache_beam/pipeline_test.py        |  399 ++
 sdks/python/apache_beam/pvalue.py               |  495 +++
 sdks/python/apache_beam/pvalue_test.py          |   67 +
 sdks/python/apache_beam/runners/__init__.py     |   29 +
 sdks/python/apache_beam/runners/common.pxd      |   75 +
 sdks/python/apache_beam/runners/common.py       |  448 ++
 .../apache_beam/runners/dataflow/__init__.py    |   16 +
 .../runners/dataflow/native_io/__init__.py      |   16 +
 .../runners/dataflow/native_io/iobase.py        |  311 ++
 .../apache_beam/runners/dataflow_runner.py      |  731 +++
 .../apache_beam/runners/dataflow_runner_test.py |   63 +
 .../apache_beam/runners/direct/__init__.py      |   19 +
 .../runners/direct/bundle_factory.py            |  102 +
 sdks/python/apache_beam/runners/direct/clock.py |   50 +
 .../consumer_tracking_pipeline_visitor.py       |   59 +
 .../consumer_tracking_pipeline_visitor_test.py  |  126 +
 .../runners/direct/direct_metrics.py            |  112 +
 .../runners/direct/direct_metrics_test.py       |  211 +
 .../apache_beam/runners/direct/direct_runner.py |  164 +
 .../runners/direct/evaluation_context.py        |  281 ++
 .../apache_beam/runners/direct/executor.py      |  568 +++
 .../runners/direct/transform_evaluator.py       |  568 +++
 .../runners/direct/transform_result.py          |   64 +
 .../runners/direct/watermark_manager.py         |  224 +
 sdks/python/apache_beam/runners/runner.py       |  351 ++
 sdks/python/apache_beam/runners/runner_test.py  |  217 +
 .../apache_beam/runners/template_runner_test.py |   88 +
 .../python/apache_beam/runners/test/__init__.py |   24 +
 .../runners/test/test_dataflow_runner.py        |   40 +
 sdks/python/apache_beam/test_pipeline.py        |  163 +
 sdks/python/apache_beam/test_pipeline_test.py   |  112 +
 sdks/python/apache_beam/tests/__init__.py       |   16 +
 sdks/python/apache_beam/tests/data/README.md    |   20 +
 .../apache_beam/tests/data/privatekey.p12       |  Bin 0 -> 2452 bytes
 .../apache_beam/tests/pipeline_verifiers.py     |  119 +
 .../tests/pipeline_verifiers_test.py            |  118 +
 sdks/python/apache_beam/tests/test_utils.py     |   57 +
 sdks/python/apache_beam/transforms/__init__.py  |   26 +
 .../python/apache_beam/transforms/aggregator.py |  120 +
 .../apache_beam/transforms/aggregator_test.py   |   77 +
 sdks/python/apache_beam/transforms/combiners.py |  595 +++
 .../apache_beam/transforms/combiners_test.py    |  335 ++
 sdks/python/apache_beam/transforms/core.py      | 1473 +++++++
 .../apache_beam/transforms/cy_combiners.pxd     |   92 +
 .../apache_beam/transforms/cy_combiners.py      |  306 ++
 sdks/python/apache_beam/transforms/display.py   |  329 ++
 .../apache_beam/transforms/display_test.py      |  205 +
 .../python/apache_beam/transforms/ptransform.py |  742 ++++
 .../apache_beam/transforms/ptransform_test.py   | 1953 ++++++++
 .../python/apache_beam/transforms/sideinputs.py |  214 +
 .../apache_beam/transforms/sideinputs_test.py   |  337 ++
 sdks/python/apache_beam/transforms/timeutil.py  |  133 +
 sdks/python/apache_beam/transforms/trigger.py   |  974 ++++
 .../apache_beam/transforms/trigger_test.py      |  582 +++
 .../transforms/trigger_transcripts.yaml         |  224 +
 sdks/python/apache_beam/transforms/util.py      |  235 +
 sdks/python/apache_beam/transforms/window.py    |  355 ++
 .../apache_beam/transforms/window_test.py       |  214 +
 .../transforms/write_ptransform_test.py         |  126 +
 sdks/python/apache_beam/typehints/__init__.py   |   22 +
 sdks/python/apache_beam/typehints/decorators.py |  533 +++
 sdks/python/apache_beam/typehints/opcodes.py    |  334 ++
 .../apache_beam/typehints/trivial_inference.py  |  417 ++
 .../typehints/trivial_inference_test.py         |  151 +
 sdks/python/apache_beam/typehints/typecheck.py  |  309 ++
 .../typehints/typed_pipeline_test.py            |  251 ++
 sdks/python/apache_beam/typehints/typehints.py  | 1061 +++++
 .../apache_beam/typehints/typehints_test.py     | 1062 +++++
 sdks/python/apache_beam/utils/__init__.py       |   22 +
 sdks/python/apache_beam/utils/annotations.py    |   99 +
 .../apache_beam/utils/annotations_test.py       |  126 +
 sdks/python/apache_beam/utils/counters.pxd      |   30 +
 sdks/python/apache_beam/utils/counters.py       |  183 +
 sdks/python/apache_beam/utils/dependency.py     |  504 +++
 .../python/apache_beam/utils/dependency_test.py |  425 ++
 sdks/python/apache_beam/utils/names.py          |   82 +
 sdks/python/apache_beam/utils/path.py           |   47 +
 sdks/python/apache_beam/utils/path_test.py      |   70 +
 .../apache_beam/utils/pipeline_options.py       |  528 +++
 .../apache_beam/utils/pipeline_options_test.py  |  175 +
 .../utils/pipeline_options_validator.py         |  200 +
 .../utils/pipeline_options_validator_test.py    |  334 ++
 sdks/python/apache_beam/utils/processes.py      |   52 +
 sdks/python/apache_beam/utils/processes_test.py |  106 +
 sdks/python/apache_beam/utils/profiler.py       |   69 +
 sdks/python/apache_beam/utils/retry.py          |  200 +
 sdks/python/apache_beam/utils/retry_test.py     |  210 +
 sdks/python/apache_beam/utils/timestamp.py      |  213 +
 sdks/python/apache_beam/utils/timestamp_test.py |  168 +
 .../python/apache_beam/utils/windowed_value.pxd |   38 +
 sdks/python/apache_beam/utils/windowed_value.py |  122 +
 .../apache_beam/utils/windowed_value_test.py    |   71 +
 sdks/python/apache_beam/version.py              |   57 +
 sdks/python/pom.xml                             |  169 +
 sdks/python/run_postcommit.sh                   |  102 +
 sdks/python/run_pylint.sh                       |   53 +
 sdks/python/setup.cfg                           |   27 +
 sdks/python/setup.py                            |  146 +
 sdks/python/test_config.py                      |   44 +
 sdks/python/tox.ini                             |   38 +
 248 files changed, 64912 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/c3b97a28/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/beam/blob/c3b97a28/sdks/pom.xml
----------------------------------------------------------------------
diff --cc sdks/pom.xml
index 150e1ae,45d8df0..3d0b893
--- a/sdks/pom.xml
+++ b/sdks/pom.xml
@@@ -33,8 -33,8 +33,9 @@@
    <name>Apache Beam :: SDKs</name>
  
    <modules>
 +    <module>common</module>
      <module>java</module>
+     <module>python</module>
    </modules>
  
    <profiles>


[44/50] [abbrv] beam git commit: Add mock time to slow bigquery unit tests.

Posted by da...@apache.org.
Add mock time to slow bigquery unit tests.

Unit tests, testing retries does not need to use real time. This change
reduces the total tox time for unit tests from 235 seconds to 73 seconds
locally.


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

Branch: refs/heads/master
Commit: e02ddac308b8b1ea0bd0cb0ae4f9ba4908a50595
Parents: 475707f
Author: Ahmet Altay <al...@google.com>
Authored: Fri Jan 27 17:35:24 2017 -0800
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jan 30 12:44:55 2017 -0800

----------------------------------------------------------------------
 sdks/python/apache_beam/io/bigquery_test.py | 26 ++++++++++++++++--------
 1 file changed, 18 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/e02ddac3/sdks/python/apache_beam/io/bigquery_test.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/io/bigquery_test.py b/sdks/python/apache_beam/io/bigquery_test.py
index b8682d1..14eb035 100644
--- a/sdks/python/apache_beam/io/bigquery_test.py
+++ b/sdks/python/apache_beam/io/bigquery_test.py
@@ -539,7 +539,8 @@ class TestBigQueryReader(unittest.TestCase):
 
 class TestBigQueryWriter(unittest.TestCase):
 
-  def test_no_table_and_create_never(self):
+  @mock.patch('time.sleep', return_value=None)
+  def test_no_table_and_create_never(self, patched_time_sleep):
     client = mock.Mock()
     client.tables.Get.side_effect = HttpError(
         response={'status': '404'}, url='', content='')
@@ -572,7 +573,9 @@ class TestBigQueryWriter(unittest.TestCase):
     self.assertTrue(client.tables.Get.called)
     self.assertTrue(client.tables.Insert.called)
 
-  def test_no_table_and_create_if_needed_and_no_schema(self):
+  @mock.patch('time.sleep', return_value=None)
+  def test_no_table_and_create_if_needed_and_no_schema(
+      self, patched_time_sleep):
     client = mock.Mock()
     client.tables.Get.side_effect = HttpError(
         response={'status': '404'}, url='', content='')
@@ -587,7 +590,9 @@ class TestBigQueryWriter(unittest.TestCase):
         'Table project:dataset.table requires a schema. None can be inferred '
         'because the table does not exist.')
 
-  def test_table_not_empty_and_write_disposition_empty(self):
+  @mock.patch('time.sleep', return_value=None)
+  def test_table_not_empty_and_write_disposition_empty(
+      self, patched_time_sleep):
     client = mock.Mock()
     client.tables.Get.return_value = bigquery.Table(
         tableReference=bigquery.TableReference(
@@ -712,7 +717,8 @@ class TestBigQueryWrapper(unittest.TestCase):
     wrapper._delete_dataset('', '')
     self.assertTrue(client.datasets.Delete.called)
 
-  def test_delete_dataset_retries_fail(self):
+  @mock.patch('time.sleep', return_value=None)
+  def test_delete_dataset_retries_fail(self, patched_time_sleep):
     client = mock.Mock()
     client.datasets.Delete.side_effect = ValueError("Cannot delete")
     wrapper = beam.io.bigquery.BigQueryWrapper(client)
@@ -730,7 +736,8 @@ class TestBigQueryWrapper(unittest.TestCase):
     wrapper._delete_table('', '', '')
     self.assertTrue(client.tables.Delete.called)
 
-  def test_delete_table_retries_fail(self):
+  @mock.patch('time.sleep', return_value=None)
+  def test_delete_table_retries_fail(self, patched_time_sleep):
     client = mock.Mock()
     client.tables.Delete.side_effect = ValueError("Cannot delete")
     wrapper = beam.io.bigquery.BigQueryWrapper(client)
@@ -738,7 +745,8 @@ class TestBigQueryWrapper(unittest.TestCase):
       wrapper._delete_table('', '', '')
     self.assertTrue(client.tables.Delete.called)
 
-  def test_delete_dataset_retries_for_timeouts(self):
+  @mock.patch('time.sleep', return_value=None)
+  def test_delete_dataset_retries_for_timeouts(self, patched_time_sleep):
     client = mock.Mock()
     client.datasets.Delete.side_effect = [
         HttpError(
@@ -749,7 +757,8 @@ class TestBigQueryWrapper(unittest.TestCase):
     wrapper._delete_dataset('', '')
     self.assertTrue(client.datasets.Delete.called)
 
-  def test_delete_table_retries_for_timeouts(self):
+  @mock.patch('time.sleep', return_value=None)
+  def test_delete_table_retries_for_timeouts(self, patched_time_sleep):
     client = mock.Mock()
     client.tables.Delete.side_effect = [
         HttpError(
@@ -760,7 +769,8 @@ class TestBigQueryWrapper(unittest.TestCase):
     wrapper._delete_table('', '', '')
     self.assertTrue(client.tables.Delete.called)
 
-  def test_temporary_dataset_is_unique(self):
+  @mock.patch('time.sleep', return_value=None)
+  def test_temporary_dataset_is_unique(self, patched_time_sleep):
     client = mock.Mock()
     client.datasets.Get.return_value = bigquery.Dataset(
         datasetReference=bigquery.DatasetReference(


[29/50] [abbrv] beam git commit: Refactoring metrics infrastructure

Posted by da...@apache.org.
Refactoring metrics infrastructure


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

Branch: refs/heads/master
Commit: b148f5cc9f3e414b9cd1f605b25d50e21f626b7a
Parents: e3849af
Author: Pablo <pa...@google.com>
Authored: Mon Jan 23 17:50:21 2017 -0800
Committer: Robert Bradshaw <ro...@google.com>
Committed: Thu Jan 26 15:28:49 2017 -0800

----------------------------------------------------------------------
 sdks/python/apache_beam/metrics/execution.pxd   | 31 +++++++++
 sdks/python/apache_beam/metrics/execution.py    | 70 ++++++++++++--------
 sdks/python/apache_beam/runners/common.pxd      |  2 +
 sdks/python/apache_beam/runners/common.py       | 11 ++-
 .../apache_beam/runners/direct/executor.py      | 12 ++--
 .../runners/direct/transform_evaluator.py       | 54 ++++++++-------
 sdks/python/setup.py                            |  1 +
 7 files changed, 125 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/b148f5cc/sdks/python/apache_beam/metrics/execution.pxd
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/metrics/execution.pxd b/sdks/python/apache_beam/metrics/execution.pxd
new file mode 100644
index 0000000..d89004f
--- /dev/null
+++ b/sdks/python/apache_beam/metrics/execution.pxd
@@ -0,0 +1,31 @@
+#
+# 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.
+#
+
+cimport cython
+
+
+cdef class MetricsContainer(object):
+  cdef object step_name
+  cdef public object counters
+  cdef public object distributions
+
+
+cdef class ScopedMetricsContainer(object):
+  cpdef enter(self)
+  cpdef exit(self)
+  cdef list _stack
+  cdef MetricsContainer _container

http://git-wip-us.apache.org/repos/asf/beam/blob/b148f5cc/sdks/python/apache_beam/metrics/execution.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/metrics/execution.py b/sdks/python/apache_beam/metrics/execution.py
index 8f04b7b..3ba1735 100644
--- a/sdks/python/apache_beam/metrics/execution.py
+++ b/sdks/python/apache_beam/metrics/execution.py
@@ -98,36 +98,49 @@ class MetricResult(object):
         self.key, self.committed, self.attempted)
 
 
-class MetricsEnvironment(object):
+class _MetricsEnvironment(object):
   """Holds the MetricsContainer for every thread and other metric information.
 
   This class is not meant to be instantiated, instead being used to keep
   track of global state.
   """
-  METRICS_SUPPORTED = False
-  _METRICS_SUPPORTED_LOCK = threading.Lock()
-
-  PER_THREAD = threading.local()
+  def __init__(self):
+    self.METRICS_SUPPORTED = False
+    self._METRICS_SUPPORTED_LOCK = threading.Lock()
+    self.PER_THREAD = threading.local()
+    self.set_container_stack()
+
+  def set_container_stack(self):
+    if not hasattr(self.PER_THREAD, 'container'):
+      self.PER_THREAD.container = []
+
+  def container_stack(self):
+    self.set_container_stack()
+    return self.PER_THREAD.container
+
+  def set_metrics_supported(self, supported):
+    self.set_container_stack()
+    with self._METRICS_SUPPORTED_LOCK:
+      self.METRICS_SUPPORTED = supported
+
+  def current_container(self):
+    self.set_container_stack()
+    index = len(self.PER_THREAD.container) - 1
+    if index < 0:
+      return None
+    else:
+      return self.PER_THREAD.container[index]
 
-  @classmethod
-  def set_metrics_supported(cls, supported):
-    with cls._METRICS_SUPPORTED_LOCK:
-      cls.METRICS_SUPPORTED = supported
+  def set_current_container(self, container):
+    self.set_container_stack()
+    self.PER_THREAD.container.append(container)
 
-  @classmethod
-  def current_container(cls):
-    try:
-      return cls.PER_THREAD.container
-    except AttributeError:
-      return None
+  def unset_current_container(self):
+    self.set_container_stack()
+    self.PER_THREAD.container.pop()
 
-  @classmethod
-  def set_current_container(cls, container):
-    cls.PER_THREAD.container = container
 
-  @classmethod
-  def unset_current_container(cls):
-    del cls.PER_THREAD.container
+MetricsEnvironment = _MetricsEnvironment()
 
 
 class MetricsContainer(object):
@@ -180,16 +193,21 @@ class MetricsContainer(object):
 
 
 class ScopedMetricsContainer(object):
-  def __init__(self, container):
-    self._old_container = MetricsEnvironment.current_container()
+  def __init__(self, container=None):
+    self._stack = MetricsEnvironment.container_stack()
     self._container = container
 
+  def enter(self):
+    self._stack.append(self._container)
+
+  def exit(self):
+    self._stack.pop()
+
   def __enter__(self):
-    MetricsEnvironment.set_current_container(self._container)
-    return self._container
+    self.enter()
 
   def __exit__(self, type, value, traceback):
-    MetricsEnvironment.set_current_container(self._old_container)
+    self.exit()
 
 
 class MetricUpdates(object):

http://git-wip-us.apache.org/repos/asf/beam/blob/b148f5cc/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 10d1f96..f41b313 100644
--- a/sdks/python/apache_beam/runners/common.pxd
+++ b/sdks/python/apache_beam/runners/common.pxd
@@ -18,6 +18,7 @@
 cimport cython
 
 from apache_beam.utils.windowed_value cimport WindowedValue
+from apache_beam.metrics.execution cimport ScopedMetricsContainer
 
 
 cdef type SideOutputValue, TimestampedValue
@@ -40,6 +41,7 @@ cdef class DoFnRunner(Receiver):
   cdef object args
   cdef dict kwargs
   cdef object side_inputs
+  cdef ScopedMetricsContainer scoped_metrics_container
   cdef bint has_windowed_side_inputs
 
   cdef Receiver main_receivers

http://git-wip-us.apache.org/repos/asf/beam/blob/b148f5cc/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 3741582..cb47513 100644
--- a/sdks/python/apache_beam/runners/common.py
+++ b/sdks/python/apache_beam/runners/common.py
@@ -22,6 +22,7 @@
 import sys
 
 from apache_beam.internal import util
+from apache_beam.metrics.execution import ScopedMetricsContainer
 from apache_beam.pvalue import SideOutputValue
 from apache_beam.transforms import core
 from apache_beam.transforms import window
@@ -69,7 +70,8 @@ class DoFnRunner(Receiver):
                logging_context=None,
                # Preferred alternative to context
                # TODO(robertwb): Remove once all runners are updated.
-               state=None):
+               state=None,
+               scoped_metrics_container=None):
     """Initializes a DoFnRunner.
 
     Args:
@@ -84,10 +86,13 @@ class DoFnRunner(Receiver):
       step_name: the name of this step
       logging_context: a LoggingContext object
       state: handle for accessing DoFn state
+      scoped_metrics_container: Context switcher for metrics container
     """
     self.step_name = step_name
     self.window_fn = windowing.windowfn
     self.tagged_receivers = tagged_receivers
+    self.scoped_metrics_container = (scoped_metrics_container
+                                     or ScopedMetricsContainer())
 
     global_window = window.GlobalWindow()
 
@@ -236,6 +241,7 @@ class DoFnRunner(Receiver):
   def _invoke_bundle_method(self, method):
     try:
       self.logging_context.enter()
+      self.scoped_metrics_container.enter()
       self.context.set_element(None)
       f = getattr(self.dofn, method)
 
@@ -251,6 +257,7 @@ class DoFnRunner(Receiver):
     except BaseException as exn:
       self.reraise_augmented(exn)
     finally:
+      self.scoped_metrics_container.exit()
       self.logging_context.exit()
 
   def start(self):
@@ -262,6 +269,7 @@ class DoFnRunner(Receiver):
   def process(self, element):
     try:
       self.logging_context.enter()
+      self.scoped_metrics_container.enter()
       if self.is_new_dofn:
         self.new_dofn_process(element)
       else:
@@ -269,6 +277,7 @@ class DoFnRunner(Receiver):
     except BaseException as exn:
       self.reraise_augmented(exn)
     finally:
+      self.scoped_metrics_container.exit()
       self.logging_context.exit()
 
   def reraise_augmented(self, exn):

http://git-wip-us.apache.org/repos/asf/beam/blob/b148f5cc/sdks/python/apache_beam/runners/direct/executor.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/runners/direct/executor.py b/sdks/python/apache_beam/runners/direct/executor.py
index 7e404f8..2d4a8bd 100644
--- a/sdks/python/apache_beam/runners/direct/executor.py
+++ b/sdks/python/apache_beam/runners/direct/executor.py
@@ -27,7 +27,7 @@ import traceback
 from weakref import WeakValueDictionary
 
 from apache_beam.metrics.execution import MetricsContainer
-from apache_beam.metrics.execution import MetricsEnvironment
+from apache_beam.metrics.execution import ScopedMetricsContainer
 
 
 class ExecutorService(object):
@@ -270,7 +270,7 @@ class TransformExecutor(ExecutorService.CallableTask):
     self._call_count += 1
     assert self._call_count <= (1 + len(self._applied_transform.side_inputs))
     metrics_container = MetricsContainer(self._applied_transform.full_label)
-    MetricsEnvironment.set_current_container(metrics_container)
+    scoped_metrics_container = ScopedMetricsContainer(metrics_container)
 
     for side_input in self._applied_transform.side_inputs:
       if side_input not in self._side_input_values:
@@ -288,14 +288,16 @@ class TransformExecutor(ExecutorService.CallableTask):
 
     try:
       evaluator = self._transform_evaluator_registry.for_application(
-          self._applied_transform, self._input_bundle, side_input_values)
+          self._applied_transform, self._input_bundle,
+          side_input_values, scoped_metrics_container)
 
       if self._input_bundle:
         for value in self._input_bundle.elements:
           evaluator.process_element(value)
 
-      result = evaluator.finish_bundle()
-      result.metric_updates = metrics_container.get_cumulative()
+      with scoped_metrics_container:
+        result = evaluator.finish_bundle()
+        result.metric_updates = metrics_container.get_cumulative()
 
       if self._evaluation_context.has_cache:
         for uncommitted_bundle in result.output_bundles:

http://git-wip-us.apache.org/repos/asf/beam/blob/b148f5cc/sdks/python/apache_beam/runners/direct/transform_evaluator.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/runners/direct/transform_evaluator.py b/sdks/python/apache_beam/runners/direct/transform_evaluator.py
index e8d8c4c..13c87c5 100644
--- a/sdks/python/apache_beam/runners/direct/transform_evaluator.py
+++ b/sdks/python/apache_beam/runners/direct/transform_evaluator.py
@@ -61,7 +61,8 @@ class TransformEvaluatorRegistry(object):
     }
 
   def for_application(
-      self, applied_ptransform, input_committed_bundle, side_inputs):
+      self, applied_ptransform, input_committed_bundle,
+      side_inputs, scoped_metrics_container):
     """Returns a TransformEvaluator suitable for processing given inputs."""
     assert applied_ptransform
     assert bool(applied_ptransform.side_inputs) == bool(side_inputs)
@@ -78,7 +79,8 @@ class TransformEvaluatorRegistry(object):
           'Execution of [%s] not implemented in runner %s.' % (
               type(applied_ptransform.transform), self))
     return evaluator(self._evaluation_context, applied_ptransform,
-                     input_committed_bundle, side_inputs)
+                     input_committed_bundle, side_inputs,
+                     scoped_metrics_container)
 
   def should_execute_serially(self, applied_ptransform):
     """Returns True if this applied_ptransform should run one bundle at a time.
@@ -108,7 +110,7 @@ class _TransformEvaluator(object):
   """An evaluator of a specific application of a transform."""
 
   def __init__(self, evaluation_context, applied_ptransform,
-               input_committed_bundle, side_inputs):
+               input_committed_bundle, side_inputs, scoped_metrics_container):
     self._evaluation_context = evaluation_context
     self._applied_ptransform = applied_ptransform
     self._input_committed_bundle = input_committed_bundle
@@ -116,7 +118,9 @@ class _TransformEvaluator(object):
     self._expand_outputs()
     self._execution_context = evaluation_context.get_execution_context(
         applied_ptransform)
-    self.start_bundle()
+    self.scoped_metrics_container = scoped_metrics_container
+    with scoped_metrics_container:
+      self.start_bundle()
 
   def _expand_outputs(self):
     outputs = set()
@@ -176,14 +180,14 @@ class _BoundedReadEvaluator(_TransformEvaluator):
   MAX_ELEMENT_PER_BUNDLE = 100
 
   def __init__(self, evaluation_context, applied_ptransform,
-               input_committed_bundle, side_inputs):
+               input_committed_bundle, side_inputs, scoped_metrics_container):
     assert not input_committed_bundle
     assert not side_inputs
     self._source = applied_ptransform.transform.source
     self._source.pipeline_options = evaluation_context.pipeline_options
     super(_BoundedReadEvaluator, self).__init__(
         evaluation_context, applied_ptransform, input_committed_bundle,
-        side_inputs)
+        side_inputs, scoped_metrics_container)
 
   def finish_bundle(self):
     assert len(self._outputs) == 1
@@ -213,11 +217,11 @@ class _FlattenEvaluator(_TransformEvaluator):
   """TransformEvaluator for Flatten transform."""
 
   def __init__(self, evaluation_context, applied_ptransform,
-               input_committed_bundle, side_inputs):
+               input_committed_bundle, side_inputs, scoped_metrics_container):
     assert not side_inputs
     super(_FlattenEvaluator, self).__init__(
         evaluation_context, applied_ptransform, input_committed_bundle,
-        side_inputs)
+        side_inputs, scoped_metrics_container)
 
   def start_bundle(self):
     assert len(self._outputs) == 1
@@ -237,12 +241,12 @@ class _CreateEvaluator(_TransformEvaluator):
   """TransformEvaluator for Create transform."""
 
   def __init__(self, evaluation_context, applied_ptransform,
-               input_committed_bundle, side_inputs):
+               input_committed_bundle, side_inputs, scoped_metrics_container):
     assert not input_committed_bundle
     assert not side_inputs
     super(_CreateEvaluator, self).__init__(
         evaluation_context, applied_ptransform, input_committed_bundle,
-        side_inputs)
+        side_inputs, scoped_metrics_container)
 
   def start_bundle(self):
     assert len(self._outputs) == 1
@@ -311,10 +315,10 @@ class _ParDoEvaluator(_TransformEvaluator):
   """TransformEvaluator for ParDo transform."""
 
   def __init__(self, evaluation_context, applied_ptransform,
-               input_committed_bundle, side_inputs):
+               input_committed_bundle, side_inputs, scoped_metrics_container):
     super(_ParDoEvaluator, self).__init__(
         evaluation_context, applied_ptransform, input_committed_bundle,
-        side_inputs)
+        side_inputs, scoped_metrics_container)
 
   def start_bundle(self):
     transform = self._applied_ptransform.transform
@@ -358,12 +362,14 @@ class _ParDoEvaluator(_TransformEvaluator):
           dofn, self._applied_ptransform.full_label)
     else:
       dofn = OutputCheckWrapperDoFn(dofn, self._applied_ptransform.full_label)
-    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))
+    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),
+        scoped_metrics_container=self.scoped_metrics_container)
     self.runner.start()
 
   def process_element(self, element):
@@ -391,11 +397,11 @@ class _GroupByKeyOnlyEvaluator(_TransformEvaluator):
       self.completed = False
 
   def __init__(self, evaluation_context, applied_ptransform,
-               input_committed_bundle, side_inputs):
+               input_committed_bundle, side_inputs, scoped_metrics_container):
     assert not side_inputs
     super(_GroupByKeyOnlyEvaluator, self).__init__(
         evaluation_context, applied_ptransform, input_committed_bundle,
-        side_inputs)
+        side_inputs, scoped_metrics_container)
 
   @property
   def _is_final_bundle(self):
@@ -463,11 +469,11 @@ class _CreatePCollectionViewEvaluator(_TransformEvaluator):
   """TransformEvaluator for CreatePCollectionView transform."""
 
   def __init__(self, evaluation_context, applied_ptransform,
-               input_committed_bundle, side_inputs):
+               input_committed_bundle, side_inputs, scoped_metrics_container):
     assert not side_inputs
     super(_CreatePCollectionViewEvaluator, self).__init__(
         evaluation_context, applied_ptransform, input_committed_bundle,
-        side_inputs)
+        side_inputs, scoped_metrics_container)
 
   @property
   def _is_final_bundle(self):
@@ -509,11 +515,11 @@ class _NativeWriteEvaluator(_TransformEvaluator):
   """TransformEvaluator for _NativeWrite transform."""
 
   def __init__(self, evaluation_context, applied_ptransform,
-               input_committed_bundle, side_inputs):
+               input_committed_bundle, side_inputs, scoped_metrics_container):
     assert not side_inputs
     super(_NativeWriteEvaluator, self).__init__(
         evaluation_context, applied_ptransform, input_committed_bundle,
-        side_inputs)
+        side_inputs, scoped_metrics_container)
 
     assert applied_ptransform.transform.sink
     self._sink = applied_ptransform.transform.sink

http://git-wip-us.apache.org/repos/asf/beam/blob/b148f5cc/sdks/python/setup.py
----------------------------------------------------------------------
diff --git a/sdks/python/setup.py b/sdks/python/setup.py
index 1fd622f..37125c2 100644
--- a/sdks/python/setup.py
+++ b/sdks/python/setup.py
@@ -118,6 +118,7 @@ setuptools.setup(
         '**/*.pyx',
         'apache_beam/coders/coder_impl.py',
         'apache_beam/runners/common.py',
+        'apache_beam/metrics/execution.py',
         'apache_beam/transforms/cy_combiners.py',
         'apache_beam/utils/counters.py',
         'apache_beam/utils/windowed_value.py',


[06/50] [abbrv] beam git commit: Closes #1810

Posted by da...@apache.org.
Closes #1810


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

Branch: refs/heads/master
Commit: 894461e64b09e6d719ba3eef282cd36cea550f7b
Parents: d0474ab f68c9dc
Author: Robert Bradshaw <ro...@google.com>
Authored: Mon Jan 23 09:48:55 2017 -0800
Committer: Robert Bradshaw <ro...@google.com>
Committed: Mon Jan 23 09:48:55 2017 -0800

----------------------------------------------------------------------
 sdks/python/apache_beam/runners/common.pxd | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------



[08/50] [abbrv] beam git commit: Closes #1818

Posted by da...@apache.org.
Closes #1818


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

Branch: refs/heads/master
Commit: 01b362807724b03969775c3a17af0854bb4b29a6
Parents: 894461e 1811458
Author: Robert Bradshaw <ro...@google.com>
Authored: Mon Jan 23 14:36:00 2017 -0800
Committer: Robert Bradshaw <ro...@google.com>
Committed: Mon Jan 23 14:36:00 2017 -0800

----------------------------------------------------------------------
 sdks/python/apache_beam/runners/dataflow_runner.py | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------



[34/50] [abbrv] beam git commit: Run lint on all files in the module.

Posted by da...@apache.org.
Run lint on all files in the module.


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

Branch: refs/heads/master
Commit: e5d88101ae191b5be6b6d0bdfbdd174893937f03
Parents: 52d97e2
Author: Ahmet Altay <al...@google.com>
Authored: Fri Jan 20 17:32:29 2017 -0800
Committer: Dan Halperin <dh...@google.com>
Committed: Fri Jan 27 14:30:17 2017 -0800

----------------------------------------------------------------------
 sdks/python/apache_beam/coders/coders.py |  1 -
 sdks/python/run_pylint.sh                | 44 ++++++++-------------------
 2 files changed, 12 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/e5d88101/sdks/python/apache_beam/coders/coders.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/coders/coders.py b/sdks/python/apache_beam/coders/coders.py
index 67bbbe6..25af934 100644
--- a/sdks/python/apache_beam/coders/coders.py
+++ b/sdks/python/apache_beam/coders/coders.py
@@ -705,4 +705,3 @@ class LengthPrefixCoder(FastCoder):
 
   def __repr__(self):
     return 'LengthPrefixCoder[%r]' % self._value_coder
-

http://git-wip-us.apache.org/repos/asf/beam/blob/e5d88101/sdks/python/run_pylint.sh
----------------------------------------------------------------------
diff --git a/sdks/python/run_pylint.sh b/sdks/python/run_pylint.sh
index e865db6..fc7b244 100755
--- a/sdks/python/run_pylint.sh
+++ b/sdks/python/run_pylint.sh
@@ -16,23 +16,16 @@
 #    limitations under the License.
 #
 
-# This script will run pylint and pep8 on files that changed compared to the
-# current HEAD of the branch.
+# This script will run pylint and pep8 on all module files.
 #
 # Use "pylint apache_beam" to run pylint all files.
 # Use "pep8 apache_beam" to run pep8 all files.
 #
 # The exit-code of the script indicates success or a failure.
 
-BASE_BRANCH=python-sdk
-
 set -e
 set -o pipefail
 
-# Retrieve base branch for comparison. Travis does not fetch it by default.
-git remote set-branches --add origin $BASE_BRANCH
-git fetch
-
 # Following generated files are excluded from lint checks.
 EXCLUDED_GENERATED_FILES=(
 "apache_beam/internal/windmill_pb2.py"
@@ -45,29 +38,16 @@ EXCLUDED_GENERATED_FILES=(
 "apache_beam/internal/clients/storage/storage_v1_messages.py"
 "apache_beam/coders/proto2_coder_test_messages_pb2.py")
 
-# Get the name of the files that changed compared to the HEAD of the branch.
-# Use diff-filter to exclude deleted files. (i.e. Do not try to lint files that
-# does not exist any more.) Filter the output to .py files only. Rewrite the
-# paths relative to the sdks/python folder.
-CHANGED_FILES=$(git diff --name-only --diff-filter=ACMRTUXB origin/$BASE_BRANCH . \
-                | { grep ".py$" || true; }  \
-                | sed 's/sdks\/python\///g')
-
-FILES_TO_CHECK=""
-for file in $CHANGED_FILES;
-do
-if [[ " ${EXCLUDED_GENERATED_FILES[@]} " =~ " ${file} " ]]; then
-  echo "Excluded file " $file " from lint checks"
-else
-  FILES_TO_CHECK="$FILES_TO_CHECK $file"
-fi
+FILES_TO_IGNORE=""
+for file in "${EXCLUDED_GENERATED_FILES[@]}"; do
+  if [[ $FILES_TO_IGNORE ]]; then
+    FILES_TO_IGNORE="$FILES_TO_IGNORE, "
+  fi
+  FILES_TO_IGNORE="$FILES_TO_IGNORE$(basename $file)" 
 done
+echo "Skipping lint for generated files: $FILES_TO_IGNORE"
 
-if test "$FILES_TO_CHECK"; then
-  echo "Running pylint on changed files:"
-  pylint $FILES_TO_CHECK
-  echo "Running pep8 on changed files:"
-  pep8 $FILES_TO_CHECK
-else
-  echo "Not running pylint. No eligible files."
-fi
+echo "Running pylint:"
+pylint apache_beam --ignore-patterns="$FILES_TO_IGNORE"
+echo "Running pep8:"
+pep8 apache_beam --exclude="$FILES_TO_IGNORE"


[17/50] [abbrv] beam git commit: Removes Dataflow native text source and sink from Beam SDK.

Posted by da...@apache.org.
Removes Dataflow native text source and sink from Beam SDK.

Users should be using Beam text source and sink available in module 'textio.py' instead of this.

Also removes Dataflow native file source/sink that is only used by native text source/sink.


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

Branch: refs/heads/master
Commit: 52fc95ddebceaaf27897c4f6d5b97d08bd4b3a1e
Parents: f983123
Author: Chamikara Jayalath <ch...@google.com>
Authored: Mon Jan 23 13:23:45 2017 -0800
Committer: Robert Bradshaw <ro...@gmail.com>
Committed: Tue Jan 24 16:31:02 2017 -0800

----------------------------------------------------------------------
 sdks/python/apache_beam/io/fileio.py            | 542 +-------------
 sdks/python/apache_beam/io/fileio_test.py       | 729 +------------------
 .../runners/direct/transform_evaluator.py       |   5 -
 3 files changed, 3 insertions(+), 1273 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/52fc95dd/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 ebc4fed..52f31c6 100644
--- a/sdks/python/apache_beam/io/fileio.py
+++ b/sdks/python/apache_beam/io/fileio.py
@@ -34,12 +34,10 @@ import weakref
 from apache_beam import coders
 from apache_beam.io import gcsio
 from apache_beam.io import iobase
-from apache_beam.io import range_trackers
-from apache_beam.runners.dataflow.native_io import iobase as dataflow_io
 from apache_beam.transforms.display import DisplayDataItem
 
 
-__all__ = ['TextFileSource', 'TextFileSink']
+__all__ = ['TextFileSink']
 
 DEFAULT_SHARD_NAME_TEMPLATE = '-SSSSS-of-NNNNN'
 
@@ -111,326 +109,6 @@ class CompressionTypes(object):
     return cls.UNCOMPRESSED
 
 
-class NativeFileSource(dataflow_io.NativeSource):
-  """A source implemented by Dataflow service from a GCS or local file or files.
-
-  This class is to be only inherited by sources natively implemented by Cloud
-  Dataflow service, hence should not be sub-classed by users.
-  """
-
-  def __init__(self,
-               file_path,
-               start_offset=None,
-               end_offset=None,
-               coder=coders.BytesCoder(),
-               compression_type=CompressionTypes.AUTO,
-               mime_type='application/octet-stream'):
-    """Initialize a NativeFileSource.
-
-    Args:
-      file_path: The file path to read from as a local file path or a GCS
-        gs:// path. The path can contain glob characters (*, ?, and [...]
-        sets).
-      start_offset: The byte offset in the source file that the reader
-        should start reading. By default is 0 (beginning of file).
-      end_offset: The byte offset in the file that the reader should stop
-        reading. By default it is the end of the file.
-      compression_type: Used to handle compressed input files. Typical value
-          is CompressionTypes.AUTO, in which case the file_path's extension will
-          be used to detect the compression.
-      coder: Coder used to decode each record.
-
-    Raises:
-      TypeError: if file_path is not a string.
-
-    If the file_path contains glob characters then the start_offset and
-    end_offset must not be specified.
-
-    The 'start_offset' and 'end_offset' pair provide a mechanism to divide the
-    file into multiple pieces for individual sources. Because the offset
-    is measured by bytes, some complication arises when the offset splits in
-    the middle of a record. To avoid the scenario where two adjacent sources
-    each get a fraction of a line we adopt the following rules:
-
-    If start_offset falls inside a record (any character except the first one)
-    then the source will skip the record and start with the next one.
-
-    If end_offset falls inside a record (any character except the first one)
-    then the source will contain that entire record.
-    """
-    if not isinstance(file_path, basestring):
-      raise TypeError('%s: file_path must be a string;  got %r instead' %
-                      (self.__class__.__name__, file_path))
-
-    self.file_path = file_path
-    self.start_offset = start_offset
-    self.end_offset = end_offset
-    self.compression_type = compression_type
-    self.coder = coder
-    self.mime_type = mime_type
-
-  def display_data(self):
-    return {'file_pattern': DisplayDataItem(self.file_path,
-                                            label="File Pattern"),
-            'compression': DisplayDataItem(str(self.compression_type),
-                                           label='Compression')}
-
-  def __eq__(self, other):
-    return (self.file_path == other.file_path and
-            self.start_offset == other.start_offset and
-            self.end_offset == other.end_offset and
-            self.compression_type == other.compression_type and
-            self.coder == other.coder and self.mime_type == other.mime_type)
-
-  @property
-  def path(self):
-    return self.file_path
-
-  def reader(self):
-    return NativeFileSourceReader(self)
-
-
-class NativeFileSourceReader(dataflow_io.NativeSourceReader,
-                             coders.observable.ObservableMixin):
-  """The source reader for a NativeFileSource.
-
-  This class is to be only inherited by source readers natively implemented by
-  Cloud Dataflow service, hence should not be sub-classed by users.
-  """
-
-  def __init__(self, source):
-    super(NativeFileSourceReader, self).__init__()
-    self.source = source
-    self.start_offset = self.source.start_offset or 0
-    self.end_offset = self.source.end_offset
-    self.current_offset = self.start_offset
-
-  def __enter__(self):
-    self.file = ChannelFactory.open(
-        self.source.file_path,
-        'rb',
-        mime_type=self.source.mime_type,
-        compression_type=self.source.compression_type)
-
-    # Determine the real end_offset.
-    #
-    # If not specified or if the source is not splittable it will be the length
-    # of the file (or infinity for compressed files) as appropriate.
-    if ChannelFactory.is_compressed(self.file):
-      if not isinstance(self.source, TextFileSource):
-        raise ValueError('Unexpected compressed file for a non-TextFileSource.')
-      self.end_offset = range_trackers.OffsetRangeTracker.OFFSET_INFINITY
-
-    elif self.end_offset is None:
-      self.file.seek(0, os.SEEK_END)
-      self.end_offset = self.file.tell()
-      self.file.seek(self.start_offset)
-
-    # Initializing range tracker after self.end_offset is finalized.
-    self.range_tracker = range_trackers.OffsetRangeTracker(self.start_offset,
-                                                           self.end_offset)
-
-    # Position to the appropriate start_offset.
-    if self.start_offset > 0 and ChannelFactory.is_compressed(self.file):
-      raise ValueError(
-          'Unexpected positive start_offset (%s) for a compressed source: %s',
-          self.start_offset, self.source)
-    self.seek_to_true_start_offset()
-
-    return self
-
-  def __exit__(self, exception_type, exception_value, traceback):
-    self.file.close()
-
-  def __iter__(self):
-    if self.current_offset > 0 and ChannelFactory.is_compressed(self.file):
-      # When compression is enabled both initial and dynamic splitting should
-      # not be allowed.
-      raise ValueError(
-          'Unespected split starting at (%s) for compressed source: %s',
-          self.current_offset, self.source)
-
-    while True:
-      if not self.range_tracker.try_claim(record_start=self.current_offset):
-        # Reader has completed reading the set of records in its range. Note
-        # that the end offset of the range may be smaller than the original
-        # end offset defined when creating the reader due to reader accepting
-        # a dynamic split request from the service.
-        return
-
-      # Note that for compressed sources, delta_offsets are virtual and don't
-      # actually correspond to byte offsets in the underlying file. They
-      # nonetheless correspond to unique virtual position locations.
-      for eof, record, delta_offset in self.read_records():
-        if eof:
-          # Can't read from this source anymore and the record and delta_offset
-          # are non-sensical; hence we are done.
-          return
-        else:
-          self.notify_observers(record, is_encoded=False)
-          self.current_offset += delta_offset
-          yield record
-
-  def seek_to_true_start_offset(self):
-    """Seeks the underlying file to the appropriate start_offset that is
-       compatible with range-tracking and position models and updates
-       self.current_offset accordingly.
-    """
-    raise NotImplementedError
-
-  def read_records(self):
-    """
-      Yields information about (possibly multiple) records corresponding to
-      self.current_offset
-
-      If a read_records() invocation returns multiple results, the first record
-      must be at a split point and other records should not be at split points.
-      The first record is assumed to be at self.current_offset and the caller
-      should use the yielded delta_offsets to update self.current_offset
-      accordingly.
-
-      The yielded value is a tripplet of the form:
-        eof, record, delta_offset
-      eof: A boolean indicating whether eof has been reached, in which case
-        the contents of record and delta_offset cannot be trusted or used.
-      record: The (possibly decoded) record (ie payload) read from the
-        underlying source.
-      delta_offset: The delta_offfset (from self.current_offset) in bytes, that
-        has been consumed from the underlying source, to the starting position
-        of the next record (or EOF if no record exists).
-    """
-    raise NotImplementedError
-
-  def get_progress(self):
-    return dataflow_io.ReaderProgress(position=dataflow_io.ReaderPosition(
-        byte_offset=self.range_tracker.last_record_start))
-
-  def request_dynamic_split(self, dynamic_split_request):
-    if ChannelFactory.is_compressed(self.file):
-      # When compression is enabled both initial and dynamic splitting should be
-      # prevented. Here we prevent dynamic splitting by ignoring all dynamic
-      # split requests at the reader.
-      return
-
-    assert dynamic_split_request is not None
-    progress = dynamic_split_request.progress
-    split_position = progress.position
-    if split_position is None:
-      percent_complete = progress.percent_complete
-      if percent_complete is not None:
-        if percent_complete <= 0 or percent_complete >= 1:
-          logging.warning(
-              'FileBasedReader cannot be split since the provided percentage '
-              'of work to be completed is out of the valid range (0, '
-              '1). Requested: %r', dynamic_split_request)
-          return
-        split_position = dataflow_io.ReaderPosition()
-        split_position.byte_offset = (
-            self.range_tracker.position_at_fraction(percent_complete))
-      else:
-        logging.warning(
-            'FileBasedReader requires either a position or a percentage of '
-            'work to be complete to perform a dynamic split request. '
-            'Requested: %r', dynamic_split_request)
-        return
-
-    if self.range_tracker.try_split(split_position.byte_offset):
-      return dataflow_io.DynamicSplitResultWithPosition(split_position)
-    else:
-      return
-
-# -----------------------------------------------------------------------------
-# TextFileSource, TextFileSink.
-
-
-class TextFileSource(NativeFileSource):
-  """A source for a GCS or local text file.
-
-  Parses a text file as newline-delimited elements, by default assuming
-  UTF-8 encoding.
-
-  This implementation has only been tested to read text encoded using UTF-8 or
-  ASCII. This has not been tested for other encodings such as UTF-16 or UTF-32.
-  """
-
-  def __init__(self,
-               file_path,
-               start_offset=None,
-               end_offset=None,
-               compression_type=CompressionTypes.AUTO,
-               strip_trailing_newlines=True,
-               coder=coders.StrUtf8Coder(),
-               mime_type='text/plain'):
-    """Initialize a TextSource.
-
-    Args:
-      file_path: The file path to read from as a local file path or a GCS
-        gs:// path. The path can contain glob characters (*, ?, and [...]
-        sets).
-      start_offset: The byte offset in the source text file that the reader
-        should start reading. By default is 0 (beginning of file).
-      end_offset: The byte offset in the file that the reader should stop
-        reading. By default it is the end of the file.
-      compression_type: Used to handle compressed input files. Typical value
-          is CompressionTypes.AUTO, in which case the file_path's extension will
-          be used to detect the compression.
-      strip_trailing_newlines: Indicates whether this source should remove
-          the newline char in each line it reads before decoding that line.
-          This feature only works for ASCII and UTF-8 encoded input.
-      coder: Coder used to decode each line.
-
-    Raises:
-      TypeError: if file_path is not a string.
-
-    If the file_path contains glob characters then the start_offset and
-    end_offset must not be specified.
-
-    The 'start_offset' and 'end_offset' pair provide a mechanism to divide the
-    text file into multiple pieces for individual sources. Because the offset
-    is measured by bytes, some complication arises when the offset splits in
-    the middle of a text line. To avoid the scenario where two adjacent sources
-    each get a fraction of a line we adopt the following rules:
-
-    If start_offset falls inside a line (any character except the first one)
-    then the source will skip the line and start with the next one.
-
-    If end_offset falls inside a line (any character except the first one) then
-    the source will contain that entire line.
-    """
-    super(TextFileSource, self).__init__(
-        file_path,
-        start_offset=start_offset,
-        end_offset=end_offset,
-        coder=coder,
-        compression_type=compression_type)
-    self.strip_trailing_newlines = strip_trailing_newlines
-
-  @property
-  def format(self):
-    """Source format name required for remote execution."""
-    return 'text'
-
-  def __eq__(self, other):
-    return (super(TextFileSource, self).__eq__(other) and
-            self.strip_trailing_newlines == other.strip_trailing_newlines)
-
-  def reader(self):
-    # If a multi-file pattern was specified as a source then make sure the
-    # start/end offsets use the default values for reading the entire file.
-    if re.search(r'[*?\[\]]', self.file_path) is not None:
-      if self.start_offset is not None:
-        raise ValueError(
-            'start offset cannot be specified for a multi-file source: '
-            '%s' % self.file_path)
-      if self.end_offset is not None:
-        raise ValueError(
-            'End offset cannot be specified for a multi-file source: '
-            '%s' % self.file_path)
-      return TextMultiFileReader(self)
-    else:
-      return TextFileReader(self)
-
-
 class ChannelFactory(object):
   # TODO: Generalize into extensible framework.
 
@@ -1114,221 +792,3 @@ class TextFileSink(FileSink):
     file_handle.write(encoded_value)
     if self.append_trailing_newlines:
       file_handle.write('\n')
-
-
-class NativeFileSink(dataflow_io.NativeSink):
-  """A sink implemented by Dataflow service to a GCS or local file or files.
-
-  This class is to be only inherited by sinks natively implemented by Cloud
-  Dataflow service, hence should not be sub-classed by users.
-  """
-
-  def __init__(self,
-               file_path_prefix,
-               file_name_suffix='',
-               num_shards=0,
-               shard_name_template=None,
-               validate=True,
-               coder=coders.BytesCoder(),
-               mime_type='application/octet-stream',
-               compression_type=CompressionTypes.AUTO):
-    if not CompressionTypes.is_valid_compression_type(compression_type):
-      raise TypeError('compression_type must be CompressionType object but '
-                      'was %s' % type(compression_type))
-
-    # We initialize a file_path attribute containing just the prefix part for
-    # local runner environment. For now, sharding is not supported in the local
-    # runner and sharding options (template, num, suffix) are ignored.
-    # The attribute is also used in the worker environment when we just write
-    # to a specific file.
-    # TODO: Add support for file sharding in the local runner.
-    self.file_path = file_path_prefix
-    self.coder = coder
-    self.file_name_prefix = file_path_prefix
-    self.file_name_suffix = file_name_suffix
-    self.num_shards = num_shards
-    # TODO: Update this when the service supports more patterns.
-    self.shard_name_template = (DEFAULT_SHARD_NAME_TEMPLATE if
-                                shard_name_template is None else
-                                shard_name_template)
-    # TODO: Implement sink validation.
-    self.validate = validate
-    self.mime_type = mime_type
-    self.compression_type = compression_type
-
-  def display_data(self):
-    file_name_pattern = '{}{}{}'.format(self.file_name_prefix,
-                                        self.shard_name_template,
-                                        self.file_name_suffix)
-    return {'shards':
-            DisplayDataItem(self.num_shards,
-                            label='Number of Shards'),
-            'file_pattern':
-            DisplayDataItem(file_name_pattern,
-                            label='File Name Pattern'),
-            'compression':
-            DisplayDataItem(str(self.compression_type),
-                            label='Compression Type')}
-
-  @property
-  def path(self):
-    return self.file_path
-
-  def writer(self):
-    return NativeFileSinkWriter(self)
-
-  def __eq__(self, other):
-    return (self.file_path == other.file_path and self.coder == other.coder and
-            self.file_name_prefix == other.file_name_prefix and
-            self.file_name_suffix == other.file_name_suffix and
-            self.num_shards == other.num_shards and
-            self.shard_name_template == other.shard_name_template and
-            self.validate == other.validate and
-            self.mime_type == other.mime_type and
-            self.compression_type == other.compression_type)
-
-
-class NativeFileSinkWriter(dataflow_io.NativeSinkWriter):
-  """The sink writer for a NativeFileSink.
-
-  This class is to be only inherited by sink writers natively implemented by
-  Cloud Dataflow service, hence should not be sub-classed by users.
-  """
-
-  def __init__(self, sink):
-    self.sink = sink
-
-  def __enter__(self):
-    self.file = ChannelFactory.open(
-        self.sink.file_path,
-        'wb',
-        mime_type=self.sink.mime_type,
-        compression_type=self.sink.compression_type)
-
-    if (ChannelFactory.is_compressed(self.file) and
-        not isinstance(self.sink, NativeTextFileSink)):
-      raise ValueError(
-          'Unexpected compressed file for a non-NativeTextFileSink.')
-
-    return self
-
-  def __exit__(self, exception_type, exception_value, traceback):
-    self.file.close()
-
-  def Write(self, value):
-    self.file.write(self.sink.coder.encode(value))
-
-
-class NativeTextFileSink(NativeFileSink):
-  """A sink to a GCS or local text file or files."""
-
-  def __init__(self,
-               file_path_prefix,
-               append_trailing_newlines=True,
-               file_name_suffix='',
-               num_shards=0,
-               shard_name_template=None,
-               validate=True,
-               coder=coders.ToStringCoder(),
-               mime_type='text/plain',
-               compression_type=CompressionTypes.AUTO):
-    super(NativeTextFileSink, self).__init__(
-        file_path_prefix,
-        file_name_suffix=file_name_suffix,
-        num_shards=num_shards,
-        shard_name_template=shard_name_template,
-        validate=validate,
-        coder=coder,
-        mime_type=mime_type,
-        compression_type=compression_type)
-    self.append_trailing_newlines = append_trailing_newlines
-
-  @property
-  def format(self):
-    """Sink format name required for remote execution."""
-    return 'text'
-
-  def writer(self):
-    return TextFileWriter(self)
-
-  def __eq__(self, other):
-    return (super(NativeTextFileSink, self).__eq__(other) and
-            self.append_trailing_newlines == other.append_trailing_newlines)
-
-# -----------------------------------------------------------------------------
-# TextFileReader, TextMultiFileReader.
-
-
-class TextFileReader(NativeFileSourceReader):
-  """A reader for a text file source."""
-
-  def seek_to_true_start_offset(self):
-    if ChannelFactory.is_compressed(self.file):
-      # When compression is enabled both initial and dynamic splitting should be
-      # prevented. Here we don't perform any seeking to a different offset, nor
-      # do we update the current_offset so that the rest of the framework can
-      # properly deal with compressed files.
-      return
-
-    if self.start_offset > 0:
-      # Read one byte before. This operation will either consume a previous
-      # newline if start_offset was at the beginning of a line or consume the
-      # line if we were in the middle of it. Either way we get the read
-      # position exactly where we wanted: at the beginning of the first full
-      # line.
-      self.file.seek(self.start_offset - 1)
-      self.current_offset -= 1
-      line = self.file.readline()
-      self.notify_observers(line, is_encoded=True)
-      self.current_offset += len(line)
-
-  def read_records(self):
-    line = self.file.readline()
-    delta_offset = len(line)
-
-    if delta_offset == 0:
-      yield True, None, None  # Reached EOF.
-    else:
-      if self.source.strip_trailing_newlines:
-        line = line.rstrip('\n')
-      yield False, self.source.coder.decode(line), delta_offset
-
-
-class TextMultiFileReader(dataflow_io.NativeSourceReader):
-  """A reader for a multi-file text source."""
-
-  def __init__(self, source):
-    self.source = source
-    self.file_paths = ChannelFactory.glob(self.source.file_path)
-    if not self.file_paths:
-      raise RuntimeError('No files found for path: %s' % self.source.file_path)
-
-  def __enter__(self):
-    return self
-
-  def __exit__(self, exception_type, exception_value, traceback):
-    pass
-
-  def __iter__(self):
-    index = 0
-    for path in self.file_paths:
-      index += 1
-      logging.info('Reading from %s (%d/%d)', path, index, len(self.file_paths))
-      with TextFileSource(
-          path,
-          strip_trailing_newlines=self.source.strip_trailing_newlines,
-          coder=self.source.coder).reader() as reader:
-        for line in reader:
-          yield line
-
-# -----------------------------------------------------------------------------
-# TextFileWriter.
-
-
-class TextFileWriter(NativeFileSinkWriter):
-  """The sink writer for a TextFileSink."""
-
-  def Write(self, value):
-    super(TextFileWriter, self).Write(value)
-    if self.sink.append_trailing_newlines:
-      self.file.write('\n')

http://git-wip-us.apache.org/repos/asf/beam/blob/52fc95dd/sdks/python/apache_beam/io/fileio_test.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/io/fileio_test.py b/sdks/python/apache_beam/io/fileio_test.py
index f75bc5d..ad77dc5 100644
--- a/sdks/python/apache_beam/io/fileio_test.py
+++ b/sdks/python/apache_beam/io/fileio_test.py
@@ -18,14 +18,11 @@
 
 """Unit tests for local and GCS sources and sinks."""
 
-import bz2
 import glob
-import gzip
 import logging
 import os
 import tempfile
 import unittest
-import zlib
 
 import hamcrest as hc
 import mock
@@ -33,555 +30,12 @@ import mock
 import apache_beam as beam
 from apache_beam import coders
 from apache_beam.io import fileio
-from apache_beam.runners.dataflow.native_io import iobase as dataflow_io
 from apache_beam.test_pipeline import TestPipeline
 from apache_beam.transforms.display import DisplayData
 from apache_beam.transforms.display_test import DisplayDataItemMatcher
 
-# TODO: Add tests for file patterns (ie not just individual files) for both
-# compressed and uncompressed files.
-
-
-class TestTextFileSource(unittest.TestCase):
-
-  def create_temp_file(self, text, suffix=''):
-    temp = tempfile.NamedTemporaryFile(delete=False, suffix=suffix)
-    with temp.file as tmp:
-      tmp.write(text)
-    return temp.name
-
-  def read_with_offsets(self,
-                        input_lines,
-                        output_lines,
-                        start_offset=None,
-                        end_offset=None):
-    file_name = self.create_temp_file('\n'.join(input_lines))
-    source = fileio.TextFileSource(
-        file_path=file_name,
-        start_offset=start_offset,
-        end_offset=end_offset)
-    read_lines = []
-    with source.reader() as reader:
-      for line in reader:
-        read_lines.append(line)
-    self.assertEqual(read_lines, output_lines)
-    dd = DisplayData.create_from(source)
-    expected_items = [
-        DisplayDataItemMatcher('file_pattern', file_name),
-        DisplayDataItemMatcher('compression', 'auto')]
-    hc.assert_that(dd.items,
-                   hc.contains_inanyorder(*expected_items))
-
-  def progress_with_offsets(self,
-                            input_lines,
-                            start_offset=None,
-                            end_offset=None):
-    source = fileio.TextFileSource(
-        file_path=self.create_temp_file('\n'.join(input_lines)),
-        start_offset=start_offset,
-        end_offset=end_offset)
-    progress_record = []
-    with source.reader() as reader:
-      self.assertEqual(reader.get_progress().position.byte_offset, -1)
-      for line in reader:
-        self.assertIsNotNone(line)
-        progress_record.append(reader.get_progress().position.byte_offset)
-
-    previous = 0
-    for current in progress_record:
-      self.assertGreater(current, previous)
-      previous = current
-
-  def test_read_entire_file(self):
-    lines = ['First', 'Second', 'Third']
-    source = fileio.TextFileSource(
-        file_path=self.create_temp_file('\n'.join(lines)))
-    read_lines = []
-    with source.reader() as reader:
-      for line in reader:
-        read_lines.append(line)
-    self.assertEqual(read_lines, lines)
-
-  def test_read_entire_file_empty(self):
-    source = fileio.TextFileSource(file_path=self.create_temp_file(''))
-    read_lines = []
-    with source.reader() as reader:
-      for line in reader:
-        read_lines.append(line)
-    self.assertEqual(read_lines, [])
-
-  def test_read_entire_file_gzip(self):
-    lines = ['First', 'Second', 'Third']
-    compressor = zlib.compressobj(-1, zlib.DEFLATED, zlib.MAX_WBITS | 16)
-    data = compressor.compress('\n'.join(lines)) + compressor.flush()
-    source = fileio.TextFileSource(
-        file_path=self.create_temp_file(data),
-        compression_type=fileio.CompressionTypes.GZIP)
-    read_lines = []
-    with source.reader() as reader:
-      for line in reader:
-        read_lines.append(line)
-    self.assertEqual(read_lines, lines)
-
-  def test_read_entire_file_gzip_auto(self):
-    lines = ['First', 'Second', 'Third']
-    compressor = zlib.compressobj(-1, zlib.DEFLATED, zlib.MAX_WBITS | 16)
-    data = compressor.compress('\n'.join(lines)) + compressor.flush()
-    source = fileio.TextFileSource(file_path=self.create_temp_file(
-        data, suffix='.gz'))
-    read_lines = []
-    with source.reader() as reader:
-      for line in reader:
-        read_lines.append(line)
-    self.assertEqual(read_lines, lines)
-
-  def test_read_entire_file_gzip_empty(self):
-    compressor = zlib.compressobj(-1, zlib.DEFLATED, zlib.MAX_WBITS | 16)
-    data = compressor.compress('') + compressor.flush()
-    source = fileio.TextFileSource(
-        file_path=self.create_temp_file(data),
-        compression_type=fileio.CompressionTypes.GZIP)
-    read_lines = []
-    with source.reader() as reader:
-      for line in reader:
-        read_lines.append(line)
-    self.assertEqual(read_lines, [])
-
-  def test_read_entire_file_gzip_large(self):
-    lines = ['Line %d' % d for d in range(100 * 1000)]
-    compressor = zlib.compressobj(-1, zlib.DEFLATED, zlib.MAX_WBITS | 16)
-    data = compressor.compress('\n'.join(lines)) + compressor.flush()
-    source = fileio.TextFileSource(
-        file_path=self.create_temp_file(data),
-        compression_type=fileio.CompressionTypes.GZIP)
-    read_lines = []
-    with source.reader() as reader:
-      for line in reader:
-        read_lines.append(line)
-    self.assertEqual(read_lines, lines)
-
-  def test_read_entire_file_bzip2(self):
-    lines = ['First', 'Second', 'Third']
-    compressor = bz2.BZ2Compressor()
-    data = compressor.compress('\n'.join(lines)) + compressor.flush()
-    source = fileio.TextFileSource(
-        file_path=self.create_temp_file(data),
-        compression_type=fileio.CompressionTypes.BZIP2)
-    read_lines = []
-    with source.reader() as reader:
-      for line in reader:
-        read_lines.append(line)
-    self.assertEqual(read_lines, lines)
-
-  def test_read_entire_file_bzip2_auto(self):
-    lines = ['First', 'Second', 'Third']
-    compressor = bz2.BZ2Compressor()
-    data = compressor.compress('\n'.join(lines)) + compressor.flush()
-    source = fileio.TextFileSource(file_path=self.create_temp_file(
-        data, suffix='.bz2'))
-    read_lines = []
-    with source.reader() as reader:
-      for line in reader:
-        read_lines.append(line)
-    self.assertEqual(read_lines, lines)
-
-  def test_read_entire_file_bzip2_empty(self):
-    compressor = bz2.BZ2Compressor()
-    data = compressor.compress('') + compressor.flush()
-    source = fileio.TextFileSource(
-        file_path=self.create_temp_file(data),
-        compression_type=fileio.CompressionTypes.BZIP2)
-    read_lines = []
-    with source.reader() as reader:
-      for line in reader:
-        read_lines.append(line)
-    self.assertEqual(read_lines, [])
-
-  def test_read_entire_file_bzip2_large(self):
-    lines = ['Line %d' % d for d in range(100 * 1000)]
-    compressor = bz2.BZ2Compressor()
-    data = compressor.compress('\n'.join(lines)) + compressor.flush()
-    source = fileio.TextFileSource(
-        file_path=self.create_temp_file(data),
-        compression_type=fileio.CompressionTypes.BZIP2)
-    read_lines = []
-    with source.reader() as reader:
-      for line in reader:
-        read_lines.append(line)
-    self.assertEqual(read_lines, lines)
-
-  def test_progress_entire_file(self):
-    lines = ['First', 'Second', 'Third']
-    source = fileio.TextFileSource(
-        file_path=self.create_temp_file('\n'.join(lines)))
-    progress_record = []
-    with source.reader() as reader:
-      self.assertEqual(-1, reader.get_progress().position.byte_offset)
-      for line in reader:
-        self.assertIsNotNone(line)
-        progress_record.append(reader.get_progress().position.byte_offset)
-      self.assertEqual(13, reader.get_progress().position.byte_offset)
-
-    self.assertEqual(len(progress_record), 3)
-    self.assertEqual(progress_record, [0, 6, 13])
-
-  def test_progress_entire_file_gzip(self):
-    lines = ['First', 'Second', 'Third']
-    compressor = zlib.compressobj(-1, zlib.DEFLATED, zlib.MAX_WBITS | 16)
-    data = compressor.compress('\n'.join(lines)) + compressor.flush()
-    source = fileio.TextFileSource(
-        file_path=self.create_temp_file(data),
-        compression_type=fileio.CompressionTypes.GZIP)
-    progress_record = []
-    with source.reader() as reader:
-      self.assertEqual(-1, reader.get_progress().position.byte_offset)
-      for line in reader:
-        self.assertIsNotNone(line)
-        progress_record.append(reader.get_progress().position.byte_offset)
-      self.assertEqual(18,  # Reading the entire contents before we decide EOF.
-                       reader.get_progress().position.byte_offset)
-
-    self.assertEqual(len(progress_record), 3)
-    self.assertEqual(progress_record, [0, 6, 13])
-
-  def test_progress_entire_file_bzip2(self):
-    lines = ['First', 'Second', 'Third']
-    compressor = bz2.BZ2Compressor()
-    data = compressor.compress('\n'.join(lines)) + compressor.flush()
-    source = fileio.TextFileSource(
-        file_path=self.create_temp_file(data),
-        compression_type=fileio.CompressionTypes.BZIP2)
-    progress_record = []
-    with source.reader() as reader:
-      self.assertEqual(-1, reader.get_progress().position.byte_offset)
-      for line in reader:
-        self.assertIsNotNone(line)
-        progress_record.append(reader.get_progress().position.byte_offset)
-      self.assertEqual(18,  # Reading the entire contents before we decide EOF.
-                       reader.get_progress().position.byte_offset)
-
-    self.assertEqual(len(progress_record), 3)
-    self.assertEqual(progress_record, [0, 6, 13])
-
-  def try_splitting_reader_at(self, reader, split_request, expected_response):
-    actual_response = reader.request_dynamic_split(split_request)
-
-    if expected_response is None:
-      self.assertIsNone(actual_response)
-    else:
-      self.assertIsNotNone(actual_response.stop_position)
-      self.assertIsInstance(actual_response.stop_position,
-                            dataflow_io.ReaderPosition)
-      self.assertIsNotNone(actual_response.stop_position.byte_offset)
-      self.assertEqual(expected_response.stop_position.byte_offset,
-                       actual_response.stop_position.byte_offset)
-
-      return actual_response
-
-  def test_file_unsplittable_gzip(self):
-    lines = ['aaaa', 'bbbb', 'cccc', 'dddd', 'eeee']
-    compressor = zlib.compressobj(-1, zlib.DEFLATED, zlib.MAX_WBITS | 16)
-    data = compressor.compress('\n'.join(lines)) + compressor.flush()
-
-    with self.assertRaises(ValueError):  # Unsplittable initially.
-      source = fileio.TextFileSource(
-          file_path=self.create_temp_file(data),
-          compression_type=fileio.CompressionTypes.GZIP,
-          start_offset=1)  # Anything other than 0 will do.
-      with source.reader():
-        pass
-
-    # Unsplittable dynamically.
-    source = fileio.TextFileSource(
-        file_path=self.create_temp_file(data),
-        compression_type=fileio.CompressionTypes.GZIP)
-
-    with source.reader() as reader:
-      percents_complete = [x / 100.0 for x in range(101)]
-
-      # Cursor at beginning of file.
-      for percent_complete in percents_complete:
-        self.try_splitting_reader_at(
-            reader,
-            dataflow_io.DynamicSplitRequest(
-                dataflow_io.ReaderProgress(percent_complete=percent_complete)),
-            None)
-
-      # Cursor passed beginning of file.
-      reader_iter = iter(reader)
-      next(reader_iter)
-      next(reader_iter)
-      for percent_complete in percents_complete:
-        self.try_splitting_reader_at(
-            reader,
-            dataflow_io.DynamicSplitRequest(
-                dataflow_io.ReaderProgress(percent_complete=percent_complete)),
-            None)
-
-  def test_file_unsplittable_bzip2(self):
-    lines = ['aaaa', 'bbbb', 'cccc', 'dddd', 'eeee']
-    compressor = bz2.BZ2Compressor()
-    data = compressor.compress('\n'.join(lines)) + compressor.flush()
-
-    with self.assertRaises(ValueError):  # Unsplittable initially.
-      source = fileio.TextFileSource(
-          file_path=self.create_temp_file(data),
-          compression_type=fileio.CompressionTypes.BZIP2,
-          start_offset=1)  # Anything other than 0 will do.
-      with source.reader():
-        pass
-
-    # Unsplittable dynamically.
-    source = fileio.TextFileSource(
-        file_path=self.create_temp_file(data),
-        compression_type=fileio.CompressionTypes.BZIP2)
-    with source.reader() as reader:
-      percents_complete = [x / 100.0 for x in range(101)]
-
-      # Cursor at beginning of file.
-      for percent_complete in percents_complete:
-        self.try_splitting_reader_at(
-            reader,
-            dataflow_io.DynamicSplitRequest(
-                dataflow_io.ReaderProgress(percent_complete=percent_complete)),
-            None)
-
-      # Cursor passed beginning of file.
-      reader_iter = iter(reader)
-      next(reader_iter)
-      next(reader_iter)
-      for percent_complete in percents_complete:
-        self.try_splitting_reader_at(
-            reader,
-            dataflow_io.DynamicSplitRequest(
-                dataflow_io.ReaderProgress(percent_complete=percent_complete)),
-            None)
-
-  def test_update_stop_position_for_percent_complete(self):
-    lines = ['aaaa', 'bbbb', 'cccc', 'dddd', 'eeee']
-    source = fileio.TextFileSource(
-        file_path=self.create_temp_file('\n'.join(lines)))
-    with source.reader() as reader:
-      # Reading two lines
-      reader_iter = iter(reader)
-      next(reader_iter)
-      next(reader_iter)
-      next(reader_iter)
-
-      # Splitting at end of the range should be unsuccessful
-      self.try_splitting_reader_at(
-          reader,
-          dataflow_io.DynamicSplitRequest(
-              dataflow_io.ReaderProgress(percent_complete=0)),
-          None)
-      self.try_splitting_reader_at(
-          reader,
-          dataflow_io.DynamicSplitRequest(
-              dataflow_io.ReaderProgress(percent_complete=1)),
-          None)
-
-      # Splitting at positions on or before start offset of the last record
-      self.try_splitting_reader_at(
-          reader,
-          dataflow_io.DynamicSplitRequest(
-              dataflow_io.ReaderProgress(percent_complete=0.2)),
-          None)
-      self.try_splitting_reader_at(
-          reader,
-          dataflow_io.DynamicSplitRequest(
-              dataflow_io.ReaderProgress(percent_complete=0.4)),
-          None)
-
-      # Splitting at a position after the start offset of the last record should
-      # be successful
-      self.try_splitting_reader_at(
-          reader,
-          dataflow_io.DynamicSplitRequest(
-              dataflow_io.ReaderProgress(percent_complete=0.6)),
-          dataflow_io.DynamicSplitResultWithPosition(
-              dataflow_io.ReaderPosition(byte_offset=15)))
-
-  def test_update_stop_position_percent_complete_for_position(self):
-    lines = ['aaaa', 'bbbb', 'cccc', 'dddd', 'eeee']
-    source = fileio.TextFileSource(
-        file_path=self.create_temp_file('\n'.join(lines)))
-    with source.reader() as reader:
-      # Reading two lines
-      reader_iter = iter(reader)
-      next(reader_iter)
-      next(reader_iter)
-      next(reader_iter)
-
-      # Splitting at end of the range should be unsuccessful
-      self.try_splitting_reader_at(
-          reader,
-          dataflow_io.DynamicSplitRequest(
-              dataflow_io.ReaderProgress(position=dataflow_io.ReaderPosition(
-                  byte_offset=0))),
-          None)
-      self.try_splitting_reader_at(
-          reader,
-          dataflow_io.DynamicSplitRequest(
-              dataflow_io.ReaderProgress(position=dataflow_io.ReaderPosition(
-                  byte_offset=25))),
-          None)
-
-      # Splitting at positions on or before start offset of the last record
-      self.try_splitting_reader_at(
-          reader,
-          dataflow_io.DynamicSplitRequest(
-              dataflow_io.ReaderProgress(position=dataflow_io.ReaderPosition(
-                  byte_offset=5))),
-          None)
-      self.try_splitting_reader_at(
-          reader,
-          dataflow_io.DynamicSplitRequest(
-              dataflow_io.ReaderProgress(position=dataflow_io.ReaderPosition(
-                  byte_offset=10))),
-          None)
-
-      # Splitting at a position after the start offset of the last record should
-      # be successful
-      self.try_splitting_reader_at(
-          reader,
-          dataflow_io.DynamicSplitRequest(
-              dataflow_io.ReaderProgress(position=dataflow_io.ReaderPosition(
-                  byte_offset=15))),
-          dataflow_io.DynamicSplitResultWithPosition(
-              dataflow_io.ReaderPosition(byte_offset=15)))
-
-  def run_update_stop_position_exhaustive(self, lines, newline):
-    """An exhaustive test for dynamic splitting.
-
-    For the given set of data items, try to perform a split at all possible
-    combinations of following.
-
-    * start position
-    * original stop position
-    * updated stop position
-    * number of items read
-
-    Args:
-      lines: set of data items to be used to create the file
-      newline: separater to be used when writing give set of lines to a text
-        file.
-    """
-
-    file_path = self.create_temp_file(newline.join(lines))
-
-    total_records = len(lines)
-    total_bytes = 0
-
-    for line in lines:
-      total_bytes += len(line)
-    total_bytes += len(newline) * (total_records - 1)
-
-    for start in xrange(0, total_bytes - 1):
-      for end in xrange(start + 1, total_bytes):
-        for stop in xrange(start, end):
-          for records_to_read in range(0, total_records):
-            self.run_update_stop_position(start, end, stop, records_to_read,
-                                          file_path)
-
-  def test_update_stop_position_exhaustive(self):
-    self.run_update_stop_position_exhaustive(
-        ['aaaa', 'bbbb', 'cccc', 'dddd', 'eeee'], '\n')
-
-  def test_update_stop_position_exhaustive_with_empty_lines(self):
-    self.run_update_stop_position_exhaustive(
-        ['', 'aaaa', '', 'bbbb', 'cccc', '', 'dddd', 'eeee', ''], '\n')
-
-  def test_update_stop_position_exhaustive_windows_newline(self):
-    self.run_update_stop_position_exhaustive(
-        ['aaaa', 'bbbb', 'cccc', 'dddd', 'eeee'], '\r\n')
-
-  def test_update_stop_position_exhaustive_multi_byte(self):
-    self.run_update_stop_position_exhaustive([u'\u0d85\u0d85\u0d85\u0d85'.encode('utf-8'),
-                                              u'\u0db6\u0db6\u0db6\u0db6'.encode('utf-8'),
-                                              u'\u0d9a\u0d9a\u0d9a\u0d9a'.encode('utf-8')], '\n')
-
-  def run_update_stop_position(self, start_offset, end_offset, stop_offset,
-                               records_to_read, file_path):
-    source = fileio.TextFileSource(file_path, start_offset, end_offset)
-
-    records_of_first_split = ''
-
-    with source.reader() as reader:
-      reader_iter = iter(reader)
-      i = 0
-
-      try:
-        while i < records_to_read:
-          records_of_first_split += next(reader_iter)
-          i += 1
-      except StopIteration:
-        # Invalid case, given source does not contain this many records.
-        return
-
-      last_record_start_after_reading = reader.range_tracker.last_record_start
-
-      if stop_offset <= last_record_start_after_reading:
-        expected_split_response = None
-      elif stop_offset == start_offset or stop_offset == end_offset:
-        expected_split_response = None
-      elif records_to_read == 0:
-        expected_split_response = None  # unstarted
-      else:
-        expected_split_response = dataflow_io.DynamicSplitResultWithPosition(
-            stop_position=dataflow_io.ReaderPosition(byte_offset=stop_offset))
-
-      split_response = self.try_splitting_reader_at(
-          reader,
-          dataflow_io.DynamicSplitRequest(progress=dataflow_io.ReaderProgress(
-              dataflow_io.ReaderPosition(byte_offset=stop_offset))),
-          expected_split_response)
-
-      # Reading remaining records from the updated reader.
-      for line in reader:
-        records_of_first_split += line
-
-    if split_response is not None:
-      # Total contents received by reading the two splits should be equal to the
-      # result obtained by reading the original source.
-      records_of_original = ''
-      records_of_second_split = ''
-
-      with source.reader() as original_reader:
-        for line in original_reader:
-          records_of_original += line
-
-      new_source = fileio.TextFileSource(
-          file_path, split_response.stop_position.byte_offset, end_offset)
-      with new_source.reader() as reader:
-        for line in reader:
-          records_of_second_split += line
-
-      self.assertEqual(records_of_original,
-                       records_of_first_split + records_of_second_split)
-
-  def test_various_offset_combination_with_local_file_for_read(self):
-    lines = ['01234', '6789012', '456789012']
-    self.read_with_offsets(lines, lines[1:], start_offset=5)
-    self.read_with_offsets(lines, lines[1:], start_offset=6)
-    self.read_with_offsets(lines, lines[2:], start_offset=7)
-    self.read_with_offsets(lines, lines[1:2], start_offset=5, end_offset=13)
-    self.read_with_offsets(lines, lines[1:2], start_offset=5, end_offset=14)
-    self.read_with_offsets(lines, lines[1:], start_offset=5, end_offset=16)
-    self.read_with_offsets(lines, lines[2:], start_offset=14, end_offset=20)
-    self.read_with_offsets(lines, lines[2:], start_offset=14)
-    self.read_with_offsets(lines, [], start_offset=20, end_offset=20)
-
-  def test_various_offset_combination_with_local_file_for_progress(self):
-    lines = ['01234', '6789012', '456789012']
-    self.progress_with_offsets(lines, start_offset=5)
-    self.progress_with_offsets(lines, start_offset=6)
-    self.progress_with_offsets(lines, start_offset=7)
-    self.progress_with_offsets(lines, start_offset=5, end_offset=13)
-    self.progress_with_offsets(lines, start_offset=5, end_offset=14)
-    self.progress_with_offsets(lines, start_offset=5, end_offset=16)
-    self.progress_with_offsets(lines, start_offset=14, end_offset=20)
-    self.progress_with_offsets(lines, start_offset=14)
-    self.progress_with_offsets(lines, start_offset=20, end_offset=20)
+
+class TestChannelFactory(unittest.TestCase):
 
   @mock.patch('apache_beam.io.fileio.gcsio')
   def test_size_of_files_in_glob_complete(self, *unused_args):
@@ -625,185 +79,6 @@ class TestTextFileSource(unittest.TestCase):
     gcsio_mock.size.assert_called_once_with('gs://bucket/file2')
 
 
-class TestNativeTextFileSink(unittest.TestCase):
-
-  def setUp(self):
-    self.lines = ['Line %d' % d for d in range(100)]
-    self.path = tempfile.NamedTemporaryFile().name
-
-  def _write_lines(self, sink, lines):
-    with sink.writer() as writer:
-      for line in lines:
-        writer.Write(line)
-
-  def test_write_text_file(self):
-    sink = fileio.NativeTextFileSink(self.path)
-    self._write_lines(sink, self.lines)
-
-    with open(self.path, 'r') as f:
-      self.assertEqual(f.read().splitlines(), self.lines)
-
-  def test_text_file_display_data(self):
-    sink = fileio.NativeTextFileSink(self.path)
-    dd = DisplayData.create_from(sink)
-    expected_items = [
-        DisplayDataItemMatcher(
-            'file_pattern',
-            '{}{}'.format(self.path, '-SSSSS-of-NNNNN')),
-        DisplayDataItemMatcher(
-            'compression',
-            'auto'),
-        DisplayDataItemMatcher(
-            'shards',
-            0)]
-    hc.assert_that(dd.items, hc.contains_inanyorder(*expected_items))
-
-  def test_text_file_display_data_suffix(self):
-    sink = fileio.NativeTextFileSink(self.path, file_name_suffix='.pdf')
-    dd = DisplayData.create_from(sink)
-    expected_items = [
-        DisplayDataItemMatcher(
-            'file_pattern',
-            '{}{}{}'.format(self.path, '-SSSSS-of-NNNNN', '.pdf')),
-        DisplayDataItemMatcher(
-            'compression',
-            'auto'),
-        DisplayDataItemMatcher(
-            'shards',
-            0)]
-    hc.assert_that(dd.items, hc.contains_inanyorder(*expected_items))
-
-  def test_write_text_file_empty(self):
-    sink = fileio.NativeTextFileSink(self.path)
-    self._write_lines(sink, [])
-
-    with open(self.path, 'r') as f:
-      self.assertEqual(f.read().splitlines(), [])
-
-  def test_write_text_gzip_file(self):
-    sink = fileio.NativeTextFileSink(
-        self.path, compression_type=fileio.CompressionTypes.GZIP)
-    self._write_lines(sink, self.lines)
-
-    with gzip.GzipFile(self.path, 'r') as f:
-      self.assertEqual(f.read().splitlines(), self.lines)
-
-  def test_display_data_gzip_file(self):
-    sink = fileio.NativeTextFileSink(
-        self.path, compression_type=fileio.CompressionTypes.GZIP)
-    dd = DisplayData.create_from(sink)
-    expected_items = [
-        DisplayDataItemMatcher(
-            'file_pattern',
-            '{}{}'.format(self.path, '-SSSSS-of-NNNNN')),
-        DisplayDataItemMatcher(
-            'compression',
-            'gzip'),
-        DisplayDataItemMatcher(
-            'shards',
-            0)]
-    hc.assert_that(dd.items, hc.contains_inanyorder(*expected_items))
-
-  def test_write_text_gzip_file_auto(self):
-    self.path = tempfile.NamedTemporaryFile(suffix='.gz').name
-    sink = fileio.NativeTextFileSink(self.path)
-    self._write_lines(sink, self.lines)
-
-    with gzip.GzipFile(self.path, 'r') as f:
-      self.assertEqual(f.read().splitlines(), self.lines)
-
-  def test_write_text_gzip_file_empty(self):
-    sink = fileio.NativeTextFileSink(
-        self.path, compression_type=fileio.CompressionTypes.GZIP)
-    self._write_lines(sink, [])
-
-    with gzip.GzipFile(self.path, 'r') as f:
-      self.assertEqual(f.read().splitlines(), [])
-
-  def test_write_text_bzip2_file(self):
-    sink = fileio.NativeTextFileSink(
-        self.path, compression_type=fileio.CompressionTypes.BZIP2)
-    self._write_lines(sink, self.lines)
-
-    with bz2.BZ2File(self.path, 'r') as f:
-      self.assertEqual(f.read().splitlines(), self.lines)
-
-  def test_display_data_bzip2_file(self):
-    sink = fileio.NativeTextFileSink(
-        self.path, compression_type=fileio.CompressionTypes.BZIP2)
-    dd = DisplayData.create_from(sink)
-    expected_items = [
-        DisplayDataItemMatcher(
-            'file_pattern',
-            '{}{}'.format(self.path, '-SSSSS-of-NNNNN')),
-        DisplayDataItemMatcher(
-            'compression',
-            'bzip2'),
-        DisplayDataItemMatcher(
-            'shards',
-            0)]
-    hc.assert_that(dd.items, hc.contains_inanyorder(*expected_items))
-
-  def test_write_text_bzip2_file_auto(self):
-    self.path = tempfile.NamedTemporaryFile(suffix='.bz2').name
-    sink = fileio.NativeTextFileSink(self.path)
-    self._write_lines(sink, self.lines)
-
-    with bz2.BZ2File(self.path, 'r') as f:
-      self.assertEqual(f.read().splitlines(), self.lines)
-
-  def test_write_text_bzip2_file_empty(self):
-    sink = fileio.NativeTextFileSink(
-        self.path, compression_type=fileio.CompressionTypes.BZIP2)
-    self._write_lines(sink, [])
-
-    with bz2.BZ2File(self.path, 'r') as f:
-      self.assertEqual(f.read().splitlines(), [])
-
-  def test_write_native(self):
-    pipeline = TestPipeline()
-    pcoll = pipeline | beam.core.Create(self.lines)
-    pcoll | beam.Write(fileio.NativeTextFileSink(self.path))  # pylint: disable=expression-not-assigned
-    pipeline.run()
-
-    read_result = []
-    for file_name in glob.glob(self.path + '*'):
-      with open(file_name, 'r') as f:
-        read_result.extend(f.read().splitlines())
-
-    self.assertEqual(read_result, self.lines)
-
-  def test_write_native_auto_compression(self):
-    pipeline = TestPipeline()
-    pcoll = pipeline | beam.core.Create(self.lines)
-    pcoll | beam.Write(  # pylint: disable=expression-not-assigned
-        fileio.NativeTextFileSink(
-            self.path, file_name_suffix='.gz'))
-    pipeline.run()
-
-    read_result = []
-    for file_name in glob.glob(self.path + '*'):
-      with gzip.GzipFile(file_name, 'r') as f:
-        read_result.extend(f.read().splitlines())
-
-    self.assertEqual(read_result, self.lines)
-
-  def test_write_native_auto_compression_unsharded(self):
-    pipeline = TestPipeline()
-    pcoll = pipeline | beam.core.Create(self.lines)
-    pcoll | beam.Write(  # pylint: disable=expression-not-assigned
-        fileio.NativeTextFileSink(
-            self.path + '.gz', shard_name_template=''))
-    pipeline.run()
-
-    read_result = []
-    for file_name in glob.glob(self.path + '*'):
-      with gzip.GzipFile(file_name, 'r') as f:
-        read_result.extend(f.read().splitlines())
-
-    self.assertEqual(read_result, self.lines)
-
-
 class MyFileSink(fileio.FileSink):
 
   def open(self, temp_path):

http://git-wip-us.apache.org/repos/asf/beam/blob/52fc95dd/sdks/python/apache_beam/runners/direct/transform_evaluator.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/runners/direct/transform_evaluator.py b/sdks/python/apache_beam/runners/direct/transform_evaluator.py
index ec2b3a1..e8d8c4c 100644
--- a/sdks/python/apache_beam/runners/direct/transform_evaluator.py
+++ b/sdks/python/apache_beam/runners/direct/transform_evaluator.py
@@ -548,11 +548,6 @@ class _NativeWriteEvaluator(_TransformEvaluator):
         # Ignore empty bundles that arrive after the output is produced.
         assert self.state == []
       else:
-        if isinstance(self._sink, io.fileio.NativeTextFileSink):
-          assert self._sink.num_shards in (0, 1)
-          if self._sink.shard_name_template:
-            self._sink.file_path += '-00000-of-00001'
-            self._sink.file_path += self._sink.file_name_suffix
         self._sink.pipeline_options = self._evaluation_context.pipeline_options
         with self._sink.writer() as writer:
           for v in self.state:


[46/50] [abbrv] beam git commit: Revert python-sdk only changes in travis, and clean incubator keywords.

Posted by da...@apache.org.
Revert python-sdk only changes in travis, and clean incubator keywords.


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

Branch: refs/heads/master
Commit: 0b4ee73a36f47a8f1b5c7ece775eae2c68af4245
Parents: 1390699
Author: Ahmet Altay <al...@google.com>
Authored: Mon Jan 30 12:51:15 2017 -0800
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jan 30 13:25:32 2017 -0800

----------------------------------------------------------------------
 .travis.yml                                          | 15 +++++++++++++++
 .../examples/cookbook/datastore_wordcount.py         |  2 +-
 sdks/python/setup.py                                 |  4 ++--
 3 files changed, 18 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/0b4ee73a/.travis.yml
----------------------------------------------------------------------
diff --git a/.travis.yml b/.travis.yml
index cb6f790..a392f7d 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -39,6 +39,20 @@ env:
 
 matrix:
   include:
+    # On OSX, run with default JDK only.
+    - os: osx
+
+    # On Linux, run with specific JDKs only.
+    - os: linux
+      env: CUSTOM_JDK="oraclejdk8" MAVEN_OVERRIDE="$MAVEN_OVERRIDE $MAVEN_CONTAINER_OVERRIDE"
+    - os: linux
+      env: CUSTOM_JDK="oraclejdk7" MAVEN_OVERRIDE="$MAVEN_OVERRIDE $MAVEN_CONTAINER_OVERRIDE"
+    - os: linux
+      env: CUSTOM_JDK="openjdk7" MAVEN_OVERRIDE="$MAVEN_OVERRIDE $MAVEN_CONTAINER_OVERRIDE"
+    - os: linux
+      env: MAVEN_OVERRIDE="-Peclipse-jdt -DskipTests $MAVEN_OVERRIDE $MAVEN_CONTAINER_OVERRIDE" CUSTOM_JDK="oraclejdk8"
+
+    # Python SDK tests.
     - os: osx
       env: TEST_PYTHON="1"
     - os: linux
@@ -51,6 +65,7 @@ before_install:
   - cat ~/.mavenrc
   - if [ "$TRAVIS_OS_NAME" == "osx" ]; then export JAVA_HOME=$(/usr/libexec/java_home); fi
   - if [ "$TRAVIS_OS_NAME" == "linux" ]; then jdk_switcher use "$CUSTOM_JDK"; fi
+  - export BEAM_SUREFIRE_ARGLINE="-Xmx512m"
   # Python SDK environment settings.
   - export TOX_ENV=py27
   - if [ "$TRAVIS_OS_NAME" == "osx" ]; then export TOX_HOME=$HOME/Library/Python/2.7/bin; fi

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4ee73a/sdks/python/apache_beam/examples/cookbook/datastore_wordcount.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/examples/cookbook/datastore_wordcount.py b/sdks/python/apache_beam/examples/cookbook/datastore_wordcount.py
index 25abb3e..067cb80 100644
--- a/sdks/python/apache_beam/examples/cookbook/datastore_wordcount.py
+++ b/sdks/python/apache_beam/examples/cookbook/datastore_wordcount.py
@@ -23,7 +23,7 @@ Cloud Datastore operations.
 
 See https://developers.google.com/datastore/ for more details on Google Cloud
 Datastore.
-See http://beam.incubator.apache.org/get-started/quickstart on
+See https://beam.apache.org/get-started/quickstart on
 how to run a Beam pipeline.
 
 Read-only Mode: In this mode, this example reads Cloud Datastore entities using

http://git-wip-us.apache.org/repos/asf/beam/blob/0b4ee73a/sdks/python/setup.py
----------------------------------------------------------------------
diff --git a/sdks/python/setup.py b/sdks/python/setup.py
index 37125c2..e75a583 100644
--- a/sdks/python/setup.py
+++ b/sdks/python/setup.py
@@ -37,10 +37,10 @@ def get_version():
 PACKAGE_NAME = 'apache-beam-sdk'
 PACKAGE_VERSION = get_version()
 PACKAGE_DESCRIPTION = 'Apache Beam SDK for Python'
-PACKAGE_URL = 'https://beam.incubator.apache.org'
+PACKAGE_URL = 'https://beam.apache.org'
 PACKAGE_DOWNLOAD_URL = 'TBD'
 PACKAGE_AUTHOR = 'Apache Software Foundation'
-PACKAGE_EMAIL = 'dev@beam.incubator.apache.org'
+PACKAGE_EMAIL = 'dev@beam.apache.org'
 PACKAGE_KEYWORDS = 'apache beam'
 PACKAGE_LONG_DESCRIPTION = '''
 TBD


[38/50] [abbrv] beam git commit: Closes #1861

Posted by da...@apache.org.
Closes #1861


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

Branch: refs/heads/master
Commit: 27cf68ee72bd58475c170712f7afe20102601606
Parents: 1bc6859 f1b8679
Author: Dan Halperin <dh...@google.com>
Authored: Sun Jan 29 08:21:18 2017 -0800
Committer: Dan Halperin <dh...@google.com>
Committed: Sun Jan 29 08:21:18 2017 -0800

----------------------------------------------------------------------
 .jenkins/common_job_properties.groovy           |    9 +-
 ...job_beam_PostCommit_Java_MavenInstall.groovy |    2 +-
 .../job_beam_PreCommit_Java_MavenInstall.groovy |    2 +-
 .../job_beam_Release_NightlySnapshot.groovy     |    2 +-
 .jenkins/job_seed.groovy                        |    2 +-
 .travis/README.md                               |    2 +-
 DISCLAIMER                                      |   10 -
 NOTICE                                          |    4 +-
 README.md                                       |   46 +-
 examples/java/README.md                         |   16 +-
 examples/java/pom.xml                           |   21 +-
 .../beam/examples/DebuggingWordCount.java       |    4 +-
 .../org/apache/beam/examples/WordCount.java     |    6 +-
 .../beam/examples/complete/AutoComplete.java    |    2 +-
 .../org/apache/beam/examples/complete/README.md |   14 +-
 .../apache/beam/examples/complete/TfIdf.java    |    2 +-
 .../examples/complete/TopWikipediaSessions.java |    2 +-
 .../examples/complete/TrafficMaxLaneFlow.java   |    2 +-
 .../beam/examples/complete/TrafficRoutes.java   |    2 +-
 .../examples/cookbook/BigQueryTornadoes.java    |    2 +-
 .../cookbook/CombinePerKeyExamples.java         |    2 +-
 .../org/apache/beam/examples/cookbook/README.md |   14 +-
 .../beam/examples/cookbook/TriggerExample.java  |    4 +-
 .../beam/examples/WindowedWordCountIT.java      |   16 +-
 examples/java8/pom.xml                          |    2 +-
 .../beam/examples/complete/game/GameStats.java  |    7 +-
 .../examples/complete/game/LeaderBoard.java     |    5 +-
 .../beam/examples/complete/game/UserScore.java  |    2 +-
 examples/pom.xml                                |   16 +-
 pom.xml                                         |   41 +-
 runners/apex/README.md                          |    4 +-
 runners/apex/pom.xml                            |    3 +-
 .../beam/runners/apex/ApexPipelineOptions.java  |    7 +-
 .../apache/beam/runners/apex/ApexRunner.java    |   43 +-
 .../beam/runners/apex/ApexYarnLauncher.java     |   23 +-
 .../translation/CreateValuesTranslator.java     |   18 +-
 .../FlattenPCollectionTranslator.java           |   28 +-
 .../apex/translation/GroupByKeyTranslator.java  |    2 +-
 .../translation/ParDoBoundMultiTranslator.java  |   27 +-
 .../apex/translation/ParDoBoundTranslator.java  |    4 +-
 .../apex/translation/TranslationContext.java    |   27 +-
 .../apex/translation/WindowBoundTranslator.java |    8 +-
 .../operators/ApexGroupByKeyOperator.java       |    4 +-
 .../operators/ApexParDoOperator.java            |    6 +-
 .../ApexReadUnboundedInputOperator.java         |   17 +-
 .../beam/runners/apex/ApexRunnerTest.java       |   75 ++
 .../beam/runners/apex/ApexYarnLauncherTest.java |    9 +-
 .../runners/apex/examples/WordCountTest.java    |    2 +-
 .../translation/ParDoBoundTranslatorTest.java   |    6 +-
 .../translation/ReadUnboundTranslatorTest.java  |    8 +-
 .../utils/ApexStateInternalsTest.java           |    2 +-
 .../test/resources/beam-runners-apex.properties |   20 +
 runners/core-java/pom.xml                       |    2 +-
 .../beam/runners/core/AssignWindowsDoFn.java    |    3 +-
 .../apache/beam/runners/core/DoFnAdapters.java  |  343 ++++++
 .../apache/beam/runners/core/DoFnRunner.java    |   21 -
 .../apache/beam/runners/core/DoFnRunners.java   |  138 +--
 .../core/GroupAlsoByWindowViaWindowSetDoFn.java |   10 +-
 .../runners/core/GroupAlsoByWindowsDoFn.java    |    5 +-
 .../beam/runners/core/KeyedWorkItemCoder.java   |    4 +-
 .../core/LateDataDroppingDoFnRunner.java        |    1 -
 .../apache/beam/runners/core/NonEmptyPanes.java |    2 +-
 .../org/apache/beam/runners/core/OldDoFn.java   |  472 ++++++++
 .../runners/core/PerKeyCombineFnRunner.java     |   70 --
 .../runners/core/PerKeyCombineFnRunners.java    |  101 --
 .../beam/runners/core/SimpleDoFnRunner.java     |   63 -
 .../beam/runners/core/SimpleOldDoFnRunner.java  |    7 +-
 .../beam/runners/core/SplittableParDo.java      |    7 -
 .../core/UnboundedReadFromBoundedSource.java    |   14 +-
 .../AfterDelayFromFirstElementStateMachine.java |    2 +-
 .../core/triggers/AfterPaneStateMachine.java    |    2 +-
 .../core/DoFnDelegatingAggregatorTest.java      |  144 +++
 .../core/GroupAlsoByWindowsProperties.java      |    2 +-
 .../runners/core/KeyedWorkItemCoderTest.java    |    6 +
 .../core/LateDataDroppingDoFnRunnerTest.java    |    2 +-
 .../apache/beam/runners/core/NoOpOldDoFn.java   |   72 ++
 .../beam/runners/core/OldDoFnContextTest.java   |   72 ++
 .../apache/beam/runners/core/OldDoFnTest.java   |  192 +++
 .../beam/runners/core/ReduceFnRunnerTest.java   |   12 +-
 .../beam/runners/core/ReduceFnTester.java       |    2 +-
 .../runners/core/SimpleOldDoFnRunnerTest.java   |    2 +-
 .../UnboundedReadFromBoundedSourceTest.java     |   12 +-
 runners/direct-java/pom.xml                     |    3 +-
 .../direct/BoundedReadEvaluatorFactory.java     |   10 +-
 ...ecycleManagerRemovingTransformEvaluator.java |   19 +-
 .../beam/runners/direct/EvaluationContext.java  |    2 +-
 .../direct/ExecutorServiceParallelExecutor.java |    4 +-
 .../runners/direct/FlattenEvaluatorFactory.java |    4 +-
 .../GroupAlsoByWindowEvaluatorFactory.java      |    9 +-
 .../direct/GroupByKeyOnlyEvaluatorFactory.java  |   10 +-
 .../beam/runners/direct/ParDoEvaluator.java     |   16 +-
 .../runners/direct/ParDoEvaluatorFactory.java   |   19 +-
 .../direct/ParDoMultiOverrideFactory.java       |   13 +-
 .../direct/StatefulParDoEvaluatorFactory.java   |   27 +-
 .../direct/TestStreamEvaluatorFactory.java      |    5 +-
 .../direct/UnboundedReadEvaluatorFactory.java   |   22 +-
 .../runners/direct/ViewEvaluatorFactory.java    |    8 +-
 .../beam/runners/direct/WatermarkManager.java   |    6 +-
 .../runners/direct/WindowEvaluatorFactory.java  |    3 +-
 .../runners/direct/AggregatorContainerTest.java |   16 +-
 .../direct/BoundedReadEvaluatorFactoryTest.java |    5 -
 .../CopyOnAccessInMemoryStateInternalsTest.java |    4 +-
 .../runners/direct/DirectGraphVisitorTest.java  |   16 +-
 .../beam/runners/direct/DirectRunnerTest.java   |    5 -
 ...leManagerRemovingTransformEvaluatorTest.java |  103 +-
 .../runners/direct/EvaluationContextTest.java   |    6 +-
 .../beam/runners/direct/ParDoEvaluatorTest.java |    3 +-
 .../StatefulParDoEvaluatorFactoryTest.java      |    4 +-
 .../UnboundedReadEvaluatorFactoryTest.java      |   13 +-
 runners/flink/README.md                         |    6 +-
 runners/flink/examples/pom.xml                  |    2 +-
 .../beam/runners/flink/examples/WordCount.java  |    2 +-
 .../flink/examples/streaming/AutoComplete.java  |    2 +-
 .../examples/streaming/KafkaIOExamples.java     |    4 +-
 .../KafkaWindowedWordCountExample.java          |    2 +-
 .../examples/streaming/WindowedWordCount.java   |    2 +-
 runners/flink/pom.xml                           |    2 +-
 runners/flink/runner/pom.xml                    |    4 +-
 .../runners/flink/FlinkPipelineOptions.java     |    6 +-
 .../runners/flink/OldPerKeyCombineFnRunner.java |   62 +
 .../flink/OldPerKeyCombineFnRunners.java        |  155 +++
 .../FlinkBatchTransformTranslators.java         |   40 +-
 .../FlinkBatchTranslationContext.java           |   21 +-
 .../FlinkStreamingTransformTranslators.java     |   46 +-
 .../FlinkStreamingTranslationContext.java       |   20 +-
 .../functions/FlinkDoFnFunction.java            |    4 +-
 .../FlinkMergingNonShuffleReduceFunction.java   |   10 +-
 .../FlinkMergingPartialReduceFunction.java      |    8 +-
 .../functions/FlinkMergingReduceFunction.java   |    8 +-
 .../functions/FlinkMultiOutputDoFnFunction.java |    4 +-
 .../FlinkMultiOutputProcessContext.java         |    2 +-
 .../functions/FlinkNoElementAssignContext.java  |    2 +-
 .../functions/FlinkPartialReduceFunction.java   |   10 +-
 .../functions/FlinkProcessContextBase.java      |    4 +-
 .../functions/FlinkReduceFunction.java          |   10 +-
 .../FlinkSingleOutputProcessContext.java        |    2 +-
 .../wrappers/streaming/DoFnOperator.java        |    6 +-
 .../streaming/SingletonKeyedWorkItemCoder.java  |   10 +-
 .../wrappers/streaming/WindowDoFnOperator.java  |    2 +-
 .../streaming/io/BoundedSourceWrapper.java      |    2 +-
 .../streaming/io/UnboundedFlinkSink.java        |    6 +
 .../streaming/io/UnboundedSourceWrapper.java    |    2 +-
 .../beam/runners/flink/PipelineOptionsTest.java |   13 +
 .../streaming/FlinkStateInternalsTest.java      |    2 +-
 .../streaming/UnboundedSourceWrapperTest.java   |  464 +++----
 runners/google-cloud-dataflow-java/pom.xml      |   15 +-
 .../beam/runners/dataflow/AssignWindows.java    |   89 ++
 .../dataflow/DataflowAggregatorTransforms.java  |   79 ++
 .../dataflow/DataflowMetricUpdateExtractor.java |  109 ++
 .../runners/dataflow/DataflowPipelineJob.java   |    2 -
 .../dataflow/DataflowPipelineTranslator.java    |  510 +++-----
 .../beam/runners/dataflow/DataflowRunner.java   |  109 +-
 .../DataflowUnboundedReadFromBoundedSource.java |  547 +++++++++
 .../beam/runners/dataflow/ReadTranslator.java   |  102 ++
 .../runners/dataflow/TransformTranslator.java   |  120 ++
 .../dataflow/internal/AssignWindows.java        |   89 --
 .../dataflow/internal/CustomSources.java        |    5 -
 .../internal/DataflowAggregatorTransforms.java  |   79 --
 .../internal/DataflowMetricUpdateExtractor.java |  109 --
 .../DataflowUnboundedReadFromBoundedSource.java |  556 ---------
 .../runners/dataflow/internal/IsmFormat.java    |   20 +-
 .../dataflow/internal/ReadTranslator.java       |  107 --
 .../DataflowPipelineWorkerPoolOptions.java      |   16 +-
 .../beam/runners/dataflow/util/DoFnInfo.java    |   66 +-
 .../beam/runners/dataflow/util/GcsStager.java   |   18 +-
 .../beam/runners/dataflow/util/PackageUtil.java |  352 ++++--
 .../beam/runners/dataflow/dataflow.properties   |    6 +-
 .../dataflow/DataflowPipelineJobTest.java       |   38 +-
 .../DataflowPipelineTranslatorTest.java         |    3 +-
 .../runners/dataflow/DataflowRunnerTest.java    |    8 +-
 ...aflowUnboundedReadFromBoundedSourceTest.java |   79 ++
 ...aflowUnboundedReadFromBoundedSourceTest.java |   83 --
 .../DataflowPipelineDebugOptionsTest.java       |    2 +-
 .../options/DataflowPipelineOptionsTest.java    |    4 +-
 .../options/DataflowProfilingOptionsTest.java   |    4 +-
 .../runners/dataflow/util/PackageUtilTest.java  |   69 +-
 runners/pom.xml                                 |   16 +-
 runners/spark/README.md                         |    8 +-
 runners/spark/pom.xml                           |   29 +-
 .../spark/aggregators/NamedAggregators.java     |    4 +-
 .../coders/BeamSparkRunnerRegistrator.java      |   48 +-
 .../spark/coders/StatelessJavaSerializer.java   |   97 ++
 .../runners/spark/coders/WritableCoder.java     |    4 +-
 .../beam/runners/spark/examples/WordCount.java  |    2 +-
 .../beam/runners/spark/io/MicrobatchSource.java |    9 +-
 .../runners/spark/io/SparkUnboundedSource.java  |  127 +-
 .../spark/stateful/StateSpecFunctions.java      |   37 +-
 .../runners/spark/translation/DoFnFunction.java |   15 +-
 .../spark/translation/EvaluationContext.java    |   83 +-
 .../translation/GroupCombineFunctions.java      |    8 +-
 .../spark/translation/MultiDoFnFunction.java    |   14 +-
 .../translation/SparkAbstractCombineFn.java     |   12 +-
 .../spark/translation/SparkGlobalCombineFn.java |   13 +-
 .../translation/SparkGroupAlsoByWindowFn.java   |    2 +-
 .../spark/translation/SparkKeyedCombineFn.java  |   13 +-
 .../spark/translation/SparkPCollectionView.java |   99 ++
 .../spark/translation/SparkRuntimeContext.java  |   63 +-
 .../spark/translation/TransformTranslator.java  |   67 +-
 .../spark/translation/TranslationUtils.java     |   37 +-
 .../streaming/StreamingTransformTranslator.java |  114 +-
 .../runners/spark/util/BroadcastHelper.java     |  127 --
 .../runners/spark/util/SideInputBroadcast.java  |   77 ++
 .../spark/util/SparkSideInputReader.java        |    8 +-
 .../coders/BeamSparkRunnerRegistratorTest.java  |   57 -
 .../streaming/KafkaStreamingTest.java           |   57 +-
 .../ResumeFromCheckpointStreamingTest.java      |   20 +-
 .../streaming/utils/PAssertStreaming.java       |    4 +-
 sdks/java/build-tools/pom.xml                   |    2 +-
 .../src/main/resources/beam/findbugs-filter.xml |   26 -
 sdks/java/core/pom.xml                          |    2 +-
 .../beam/sdk/annotations/Experimental.java      |    5 +-
 .../org/apache/beam/sdk/coders/AtomicCoder.java |    2 +-
 .../org/apache/beam/sdk/coders/AvroCoder.java   |   30 +-
 .../apache/beam/sdk/coders/BigDecimalCoder.java |    6 +-
 .../beam/sdk/coders/BigEndianIntegerCoder.java  |    7 +
 .../beam/sdk/coders/BigEndianLongCoder.java     |    7 +
 .../apache/beam/sdk/coders/ByteArrayCoder.java  |    7 +
 .../org/apache/beam/sdk/coders/ByteCoder.java   |    7 +
 .../apache/beam/sdk/coders/ByteStringCoder.java |    8 +
 .../java/org/apache/beam/sdk/coders/Coder.java  |    7 +
 .../apache/beam/sdk/coders/CollectionCoder.java |   12 +-
 .../org/apache/beam/sdk/coders/CustomCoder.java |   18 +-
 .../apache/beam/sdk/coders/DelegateCoder.java   |   29 +-
 .../org/apache/beam/sdk/coders/DoubleCoder.java |    7 +
 .../apache/beam/sdk/coders/DurationCoder.java   |    8 +
 .../apache/beam/sdk/coders/InstantCoder.java    |    7 +
 .../apache/beam/sdk/coders/IterableCoder.java   |   12 +-
 .../org/apache/beam/sdk/coders/JAXBCoder.java   |   48 +-
 .../org/apache/beam/sdk/coders/KvCoder.java     |   35 +-
 .../beam/sdk/coders/LengthPrefixCoder.java      |  145 +++
 .../org/apache/beam/sdk/coders/ListCoder.java   |    7 +
 .../org/apache/beam/sdk/coders/MapCoder.java    |   62 +-
 .../apache/beam/sdk/coders/NullableCoder.java   |    6 +
 .../beam/sdk/coders/SerializableCoder.java      |   17 +-
 .../org/apache/beam/sdk/coders/SetCoder.java    |   12 +-
 .../apache/beam/sdk/coders/StandardCoder.java   |   39 +-
 .../beam/sdk/coders/StringDelegateCoder.java    |   16 +-
 .../apache/beam/sdk/coders/StringUtf8Coder.java |    7 +
 .../beam/sdk/coders/TableRowJsonCoder.java      |    7 +
 .../beam/sdk/coders/TextualIntegerCoder.java    |    8 +
 .../org/apache/beam/sdk/coders/VarIntCoder.java |   10 +-
 .../apache/beam/sdk/coders/VarLongCoder.java    |    7 +
 .../org/apache/beam/sdk/coders/VoidCoder.java   |    7 +
 .../beam/sdk/coders/protobuf/ProtoCoder.java    |    8 +-
 .../java/org/apache/beam/sdk/io/AvroSource.java |    5 -
 .../sdk/io/BoundedReadFromUnboundedSource.java  |   79 +-
 .../org/apache/beam/sdk/io/BoundedSource.java   |    8 -
 .../apache/beam/sdk/io/CompressedSource.java    |    8 -
 .../org/apache/beam/sdk/io/CountingSource.java  |    5 -
 .../org/apache/beam/sdk/io/FileSystems.java     |   32 +-
 .../java/org/apache/beam/sdk/io/PubsubIO.java   | 1142 +++++++++---------
 .../apache/beam/sdk/io/PubsubUnboundedSink.java |   88 +-
 .../beam/sdk/io/PubsubUnboundedSource.java      |  104 +-
 .../main/java/org/apache/beam/sdk/io/Read.java  |    7 +-
 .../java/org/apache/beam/sdk/io/TextIO.java     |    5 -
 .../java/org/apache/beam/sdk/io/XmlSource.java  |    5 -
 .../org/apache/beam/sdk/options/GcpOptions.java |   36 +-
 .../org/apache/beam/sdk/options/GcsOptions.java |    4 +-
 .../beam/sdk/options/PipelineOptions.java       |    2 +-
 .../sdk/options/PipelineOptionsFactory.java     |   10 +-
 .../apache/beam/sdk/options/ValueProvider.java  |    6 +-
 .../beam/sdk/runners/TransformHierarchy.java    |   33 +-
 .../testing/FlattenWithHeterogeneousCoders.java |   29 +
 .../org/apache/beam/sdk/testing/PAssert.java    |   12 +-
 .../beam/sdk/testing/RunnableOnService.java     |   14 +-
 .../beam/sdk/testing/SourceTestUtils.java       |    5 -
 .../org/apache/beam/sdk/testing/TestStream.java |    8 +
 .../sdk/testing/UsesUnboundedPCollections.java  |   23 +
 .../beam/sdk/testing/ValueInSingleWindow.java   |    6 +-
 .../sdk/transforms/AggregatorRetriever.java     |   13 +-
 .../beam/sdk/transforms/AppliedPTransform.java  |   11 +-
 .../org/apache/beam/sdk/transforms/Combine.java |  197 +--
 .../apache/beam/sdk/transforms/CombineFns.java  |   14 +-
 .../org/apache/beam/sdk/transforms/Count.java   |    4 +-
 .../org/apache/beam/sdk/transforms/Create.java  |    5 -
 .../sdk/transforms/DelegatingAggregator.java    |    2 +-
 .../beam/sdk/transforms/DoFnAdapters.java       |  504 --------
 .../apache/beam/sdk/transforms/DoFnTester.java  |    7 -
 .../apache/beam/sdk/transforms/GroupByKey.java  |    2 +-
 .../org/apache/beam/sdk/transforms/Max.java     |  124 +-
 .../org/apache/beam/sdk/transforms/Mean.java    |   27 +-
 .../org/apache/beam/sdk/transforms/Min.java     |  122 +-
 .../org/apache/beam/sdk/transforms/OldDoFn.java |  758 ------------
 .../apache/beam/sdk/transforms/PTransform.java  |    9 +-
 .../org/apache/beam/sdk/transforms/ParDo.java   |   46 +-
 .../org/apache/beam/sdk/transforms/Regex.java   |  589 ++++++++-
 .../org/apache/beam/sdk/transforms/Sum.java     |   57 +-
 .../apache/beam/sdk/transforms/ToString.java    |  198 +++
 .../org/apache/beam/sdk/transforms/Top.java     |   27 +-
 .../beam/sdk/transforms/join/CoGbkResult.java   |   35 +-
 .../sdk/transforms/reflect/DoFnInvoker.java     |   20 -
 .../sdk/transforms/reflect/DoFnInvokers.java    |  142 +--
 .../sdk/transforms/reflect/DoFnSignature.java   |   15 +-
 .../windowing/AfterDelayFromFirstElement.java   |    2 +-
 .../sdk/transforms/windowing/AfterPane.java     |    2 +-
 .../sdk/transforms/windowing/GlobalWindow.java  |    6 +
 .../transforms/windowing/IntervalWindow.java    |    4 +-
 .../beam/sdk/transforms/windowing/Window.java   |    3 +-
 .../org/apache/beam/sdk/util/CoderUtils.java    |   28 +-
 .../beam/sdk/util/CombineContextFactory.java    |   18 -
 .../org/apache/beam/sdk/util/DefaultBucket.java |  105 ++
 .../util/EmptyOnDeserializationThreadLocal.java |   39 +
 .../apache/beam/sdk/util/GcpProjectUtil.java    |    2 +-
 .../java/org/apache/beam/sdk/util/GcsUtil.java  |   36 +-
 .../org/apache/beam/sdk/util/NameUtils.java     |  162 +++
 .../org/apache/beam/sdk/util/PropertyNames.java |    1 +
 .../org/apache/beam/sdk/util/PubsubClient.java  |   28 +-
 .../apache/beam/sdk/util/PubsubGrpcClient.java  |    6 +-
 .../apache/beam/sdk/util/PubsubJsonClient.java  |    4 +-
 .../apache/beam/sdk/util/PubsubTestClient.java  |    6 +-
 .../org/apache/beam/sdk/util/StringUtils.java   |  100 --
 .../apache/beam/sdk/util/TimerInternals.java    |    4 +-
 .../org/apache/beam/sdk/util/WindowedValue.java |   23 +-
 .../beam/sdk/util/state/StateContexts.java      |    4 +-
 .../org/apache/beam/sdk/values/PValueBase.java  |    4 +-
 .../beam/sdk/values/TimestampedValue.java       |   10 +-
 .../sdk/AggregatorPipelineExtractorTest.java    |   16 +-
 .../apache/beam/sdk/coders/AvroCoderTest.java   |    7 +
 .../beam/sdk/coders/BigDecimalCoderTest.java    |   46 +-
 .../sdk/coders/BigEndianIntegerCoderTest.java   |    9 +
 .../beam/sdk/coders/BigEndianLongCoderTest.java |    9 +
 .../beam/sdk/coders/ByteArrayCoderTest.java     |    6 +
 .../apache/beam/sdk/coders/ByteCoderTest.java   |    9 +
 .../beam/sdk/coders/ByteStringCoderTest.java    |    8 +
 .../beam/sdk/coders/CoderRegistryTest.java      |    6 +
 .../org/apache/beam/sdk/coders/CoderTest.java   |    8 +
 .../beam/sdk/coders/CollectionCoderTest.java    |   16 +
 .../beam/sdk/coders/DefaultCoderTest.java       |    4 +-
 .../beam/sdk/coders/DelegateCoderTest.java      |   35 +-
 .../apache/beam/sdk/coders/DoubleCoderTest.java |    9 +
 .../beam/sdk/coders/DurationCoderTest.java      |   10 +
 .../beam/sdk/coders/InstantCoderTest.java       |    9 +
 .../beam/sdk/coders/IterableCoderTest.java      |   27 +-
 .../apache/beam/sdk/coders/JAXBCoderTest.java   |   26 +-
 .../org/apache/beam/sdk/coders/KvCoderTest.java |   29 +
 .../beam/sdk/coders/LengthPrefixCoderTest.java  |  129 ++
 .../apache/beam/sdk/coders/ListCoderTest.java   |   16 +-
 .../apache/beam/sdk/coders/MapCoderTest.java    |   21 +-
 .../beam/sdk/coders/NullableCoderTest.java      |   12 +
 .../beam/sdk/coders/SerializableCoderTest.java  |    9 +
 .../apache/beam/sdk/coders/SetCoderTest.java    |   16 +
 .../beam/sdk/coders/StandardCoderTest.java      |   40 +
 .../sdk/coders/StringDelegateCoderTest.java     |   11 +
 .../beam/sdk/coders/StringUtf8CoderTest.java    |    9 +
 .../beam/sdk/coders/TableRowJsonCoderTest.java  |    9 +
 .../sdk/coders/TextualIntegerCoderTest.java     |    9 +
 .../apache/beam/sdk/coders/VarIntCoderTest.java |    9 +
 .../beam/sdk/coders/VarLongCoderTest.java       |    9 +
 .../apache/beam/sdk/coders/VoidCoderTest.java   |   40 +
 .../beam/sdk/io/AvroIOGeneratedClassTest.java   |  285 -----
 .../apache/beam/sdk/io/AvroIOTransformTest.java |  324 +++++
 .../beam/sdk/io/CompressedSourceTest.java       |    5 -
 .../apache/beam/sdk/io/FileBasedSourceTest.java |    5 -
 .../org/apache/beam/sdk/io/FileSystemsTest.java |   33 +-
 .../beam/sdk/io/OffsetBasedSourceTest.java      |    5 -
 .../org/apache/beam/sdk/io/PubsubIOTest.java    |   86 +-
 .../beam/sdk/io/PubsubUnboundedSinkTest.java    |   41 +-
 .../beam/sdk/io/PubsubUnboundedSourceTest.java  |   10 +-
 .../java/org/apache/beam/sdk/io/ReadTest.java   |    5 -
 .../java/org/apache/beam/sdk/io/WriteTest.java  |   10 +-
 .../apache/beam/sdk/options/GcpOptionsTest.java |    4 +-
 .../sdk/options/PipelineOptionsFactoryTest.java |    6 +-
 .../beam/sdk/options/PipelineOptionsTest.java   |    3 +-
 .../beam/sdk/options/ValueProviderTest.java     |   36 +-
 .../sdk/options/ValueProviderUtilsTest.java     |    2 +-
 .../sdk/runners/TransformHierarchyTest.java     |   30 +-
 .../apache/beam/sdk/testing/TestStreamTest.java |    5 +
 .../testing/ValueInSingleWindowCoderTest.java   |    7 +
 .../sdk/transforms/ApproximateUniqueTest.java   |  483 ++++----
 .../beam/sdk/transforms/CombineFnsTest.java     |   20 +-
 .../apache/beam/sdk/transforms/CombineTest.java |   99 +-
 .../apache/beam/sdk/transforms/CountTest.java   |    2 +-
 .../apache/beam/sdk/transforms/CreateTest.java  |    8 -
 .../DoFnDelegatingAggregatorTest.java           |  142 ---
 .../apache/beam/sdk/transforms/DoFnTest.java    |   15 +-
 .../beam/sdk/transforms/DoFnTesterTest.java     |    6 +-
 .../apache/beam/sdk/transforms/FlattenTest.java |   27 +
 .../apache/beam/sdk/transforms/KvSwapTest.java  |   13 +-
 .../org/apache/beam/sdk/transforms/MaxTest.java |   20 +-
 .../apache/beam/sdk/transforms/MeanTest.java    |    7 +-
 .../org/apache/beam/sdk/transforms/MinTest.java |   21 +-
 .../apache/beam/sdk/transforms/NoOpOldDoFn.java |   71 --
 .../beam/sdk/transforms/OldDoFnContextTest.java |   69 --
 .../apache/beam/sdk/transforms/OldDoFnTest.java |  188 ---
 .../apache/beam/sdk/transforms/ParDoTest.java   |   74 +-
 .../apache/beam/sdk/transforms/RegexTest.java   |  127 +-
 .../apache/beam/sdk/transforms/SampleTest.java  |  405 ++++---
 .../beam/sdk/transforms/SimpleStatsFnsTest.java |   36 +-
 .../org/apache/beam/sdk/transforms/SumTest.java |   24 +-
 .../beam/sdk/transforms/ToStringTest.java       |  125 ++
 .../org/apache/beam/sdk/transforms/TopTest.java |   13 +-
 .../apache/beam/sdk/transforms/ViewTest.java    |    2 +-
 .../transforms/join/CoGbkResultCoderTest.java   |   10 +-
 .../sdk/transforms/join/UnionCoderTest.java     |   24 +-
 .../transforms/reflect/DoFnInvokersTest.java    |   42 -
 .../transforms/windowing/GlobalWindowTest.java  |   64 +
 .../apache/beam/sdk/util/CombineFnUtilTest.java |    8 +-
 .../apache/beam/sdk/util/DefaultBucketTest.java |  112 ++
 .../org/apache/beam/sdk/util/GcsUtilTest.java   |   56 +
 .../org/apache/beam/sdk/util/NameUtilsTest.java |  177 +++
 .../beam/sdk/util/PubsubGrpcClientTest.java     |    8 +-
 .../beam/sdk/util/PubsubJsonClientTest.java     |    3 +-
 .../beam/sdk/util/PubsubTestClientTest.java     |    4 +-
 .../beam/sdk/util/SerializableUtilsTest.java    |    4 +-
 .../apache/beam/sdk/util/StringUtilsTest.java   |  100 --
 .../beam/sdk/util/TimerInternalsTest.java       |    5 +
 .../beam/sdk/util/ValueWithRecordIdTest.java    |   34 +
 .../apache/beam/sdk/util/WindowedValueTest.java |   23 +
 .../util/state/InMemoryStateInternalsTest.java  |    2 +-
 .../beam/sdk/util/state/StateTagTest.java       |   11 +-
 .../beam/sdk/values/TimestampedValueTest.java   |   19 +-
 sdks/java/extensions/join-library/README.md     |   10 -
 sdks/java/extensions/join-library/pom.xml       |    2 +-
 sdks/java/extensions/pom.xml                    |    2 +-
 sdks/java/extensions/sorter/pom.xml             |    2 +-
 sdks/java/io/elasticsearch/pom.xml              |  175 +++
 .../sdk/io/elasticsearch/ElasticsearchIO.java   |  819 +++++++++++++
 .../beam/sdk/io/elasticsearch/package-info.java |   20 +
 .../elasticsearch/ElasticSearchIOTestUtils.java |  129 ++
 .../io/elasticsearch/ElasticsearchIOTest.java   |  358 ++++++
 sdks/java/io/google-cloud-platform/pom.xml      |    2 +-
 .../beam/sdk/io/gcp/bigquery/BigQueryIO.java    |  308 +++--
 .../sdk/io/gcp/bigquery/BigQueryServices.java   |   16 +-
 .../io/gcp/bigquery/BigQueryServicesImpl.java   |   76 +-
 .../beam/sdk/io/gcp/bigtable/BigtableIO.java    |    8 -
 .../io/gcp/bigtable/BigtableTestOptions.java    |   37 -
 .../sdk/io/gcp/bigquery/BigQueryIOTest.java     |  205 +++-
 .../gcp/bigquery/BigQueryServicesImplTest.java  |  141 +++
 .../sdk/io/gcp/bigquery/BigQueryUtilTest.java   |    3 +-
 .../sdk/io/gcp/bigtable/BigtableIOTest.java     |    5 +-
 .../io/gcp/bigtable/BigtableTestOptions.java    |   37 +
 sdks/java/io/hdfs/pom.xml                       |    2 +-
 .../beam/sdk/io/hdfs/AvroWrapperCoder.java      |    4 +-
 .../apache/beam/sdk/io/hdfs/HDFSFileSource.java |    5 -
 .../apache/beam/sdk/io/hdfs/WritableCoder.java  |    4 +-
 .../beam/sdk/io/hdfs/AvroWrapperCoderTest.java  |    1 -
 sdks/java/io/jdbc/pom.xml                       |    2 +-
 sdks/java/io/jms/pom.xml                        |    2 +-
 sdks/java/io/kafka/pom.xml                      |    2 +-
 .../org/apache/beam/sdk/io/kafka/KafkaIO.java   |    7 +-
 .../beam/sdk/io/kafka/KafkaRecordCoder.java     |    4 +-
 .../apache/beam/sdk/io/kafka/KafkaIOTest.java   |    9 +-
 .../beam/sdk/io/kafka/KafkaRecordCoderTest.java |   34 +
 sdks/java/io/kinesis/pom.xml                    |    2 +-
 .../beam/sdk/io/kinesis/KinesisRecordCoder.java |    4 +-
 .../beam/sdk/io/kinesis/package-info.java       |    2 +-
 sdks/java/io/mongodb/pom.xml                    |    2 +-
 .../beam/sdk/io/mongodb/MongoDbGridFSIO.java    |    5 -
 .../apache/beam/sdk/io/mongodb/MongoDbIO.java   |    5 -
 sdks/java/io/mqtt/pom.xml                       |  152 +++
 .../org/apache/beam/sdk/io/mqtt/MqttIO.java     |  588 +++++++++
 .../apache/beam/sdk/io/mqtt/package-info.java   |   22 +
 .../org/apache/beam/sdk/io/mqtt/MqttIOTest.java |  197 +++
 sdks/java/io/pom.xml                            |    4 +-
 sdks/java/java8tests/pom.xml                    |    2 +-
 .../maven-archetypes/examples-java8/pom.xml     |    2 +-
 .../main/resources/archetype-resources/pom.xml  |   19 +-
 sdks/java/maven-archetypes/examples/pom.xml     |    2 +-
 .../main/resources/archetype-resources/pom.xml  |   19 +-
 sdks/java/maven-archetypes/pom.xml              |    2 +-
 sdks/java/maven-archetypes/starter/pom.xml      |    2 +-
 .../main/resources/archetype-resources/pom.xml  |    4 +-
 .../resources/projects/basic/reference/pom.xml  |    4 +-
 sdks/java/pom.xml                               |    2 +-
 sdks/pom.xml                                    |   15 +-
 sdks/python/pom.xml                             |    2 +-
 466 files changed, 13883 insertions(+), 8047 deletions(-)
----------------------------------------------------------------------



[23/50] [abbrv] beam git commit: Closes #1844

Posted by da...@apache.org.
Closes #1844


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

Branch: refs/heads/master
Commit: 4e1028b3dfeaf02e51eb9f3b5d1a5e78c1cfcbb9
Parents: 5924220 5787e81
Author: Robert Bradshaw <ro...@google.com>
Authored: Wed Jan 25 16:16:52 2017 -0800
Committer: Robert Bradshaw <ro...@google.com>
Committed: Wed Jan 25 16:16:52 2017 -0800

----------------------------------------------------------------------
 .../python/apache_beam/utils/dependency_test.py | 47 +++++++++++---------
 1 file changed, 27 insertions(+), 20 deletions(-)
----------------------------------------------------------------------



[21/50] [abbrv] beam git commit: Closes #1811

Posted by da...@apache.org.
Closes #1811


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

Branch: refs/heads/master
Commit: 592422059e21bf72fc7b4842d6fd6df000a7d2a7
Parents: 9540cf1 61d8d3f
Author: Robert Bradshaw <ro...@google.com>
Authored: Wed Jan 25 12:38:03 2017 -0800
Committer: Robert Bradshaw <ro...@google.com>
Committed: Wed Jan 25 12:38:03 2017 -0800

----------------------------------------------------------------------
 sdks/python/apache_beam/pipeline_test.py | 57 ++++++++++-----------------
 1 file changed, 21 insertions(+), 36 deletions(-)
----------------------------------------------------------------------



[48/50] [abbrv] beam git commit: Remove sdks/python/LICENSE

Posted by da...@apache.org.
Remove sdks/python/LICENSE


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

Branch: refs/heads/master
Commit: e0ff9973940c5585376ac2beb6edd1e20de962ac
Parents: be0e32e
Author: Davor Bonaci <da...@google.com>
Authored: Mon Jan 30 14:41:32 2017 -0800
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jan 30 14:41:32 2017 -0800

----------------------------------------------------------------------
 sdks/python/LICENSE | 202 -----------------------------------------------
 1 file changed, 202 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/e0ff9973/sdks/python/LICENSE
----------------------------------------------------------------------
diff --git a/sdks/python/LICENSE b/sdks/python/LICENSE
deleted file mode 100644
index d645695..0000000
--- a/sdks/python/LICENSE
+++ /dev/null
@@ -1,202 +0,0 @@
-
-                                 Apache License
-                           Version 2.0, January 2004
-                        http://www.apache.org/licenses/
-
-   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
-
-   1. Definitions.
-
-      "License" shall mean the terms and conditions for use, reproduction,
-      and distribution as defined by Sections 1 through 9 of this document.
-
-      "Licensor" shall mean the copyright owner or entity authorized by
-      the copyright owner that is granting the License.
-
-      "Legal Entity" shall mean the union of the acting entity and all
-      other entities that control, are controlled by, or are under common
-      control with that entity. For the purposes of this definition,
-      "control" means (i) the power, direct or indirect, to cause the
-      direction or management of such entity, whether by contract or
-      otherwise, or (ii) ownership of fifty percent (50%) or more of the
-      outstanding shares, or (iii) beneficial ownership of such entity.
-
-      "You" (or "Your") shall mean an individual or Legal Entity
-      exercising permissions granted by this License.
-
-      "Source" form shall mean the preferred form for making modifications,
-      including but not limited to software source code, documentation
-      source, and configuration files.
-
-      "Object" form shall mean any form resulting from mechanical
-      transformation or translation of a Source form, including but
-      not limited to compiled object code, generated documentation,
-      and conversions to other media types.
-
-      "Work" shall mean the work of authorship, whether in Source or
-      Object form, made available under the License, as indicated by a
-      copyright notice that is included in or attached to the work
-      (an example is provided in the Appendix below).
-
-      "Derivative Works" shall mean any work, whether in Source or Object
-      form, that is based on (or derived from) the Work and for which the
-      editorial revisions, annotations, elaborations, or other modifications
-      represent, as a whole, an original work of authorship. For the purposes
-      of this License, Derivative Works shall not include works that remain
-      separable from, or merely link (or bind by name) to the interfaces of,
-      the Work and Derivative Works thereof.
-
-      "Contribution" shall mean any work of authorship, including
-      the original version of the Work and any modifications or additions
-      to that Work or Derivative Works thereof, that is intentionally
-      submitted to Licensor for inclusion in the Work by the copyright owner
-      or by an individual or Legal Entity authorized to submit on behalf of
-      the copyright owner. For the purposes of this definition, "submitted"
-      means any form of electronic, verbal, or written communication sent
-      to the Licensor or its representatives, including but not limited to
-      communication on electronic mailing lists, source code control systems,
-      and issue tracking systems that are managed by, or on behalf of, the
-      Licensor for the purpose of discussing and improving the Work, but
-      excluding communication that is conspicuously marked or otherwise
-      designated in writing by the copyright owner as "Not a Contribution."
-
-      "Contributor" shall mean Licensor and any individual or Legal Entity
-      on behalf of whom a Contribution has been received by Licensor and
-      subsequently incorporated within the Work.
-
-   2. Grant of Copyright License. Subject to the terms and conditions of
-      this License, each Contributor hereby grants to You a perpetual,
-      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
-      copyright license to reproduce, prepare Derivative Works of,
-      publicly display, publicly perform, sublicense, and distribute the
-      Work and such Derivative Works in Source or Object form.
-
-   3. Grant of Patent License. Subject to the terms and conditions of
-      this License, each Contributor hereby grants to You a perpetual,
-      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
-      (except as stated in this section) patent license to make, have made,
-      use, offer to sell, sell, import, and otherwise transfer the Work,
-      where such license applies only to those patent claims licensable
-      by such Contributor that are necessarily infringed by their
-      Contribution(s) alone or by combination of their Contribution(s)
-      with the Work to which such Contribution(s) was submitted. If You
-      institute patent litigation against any entity (including a
-      cross-claim or counterclaim in a lawsuit) alleging that the Work
-      or a Contribution incorporated within the Work constitutes direct
-      or contributory patent infringement, then any patent licenses
-      granted to You under this License for that Work shall terminate
-      as of the date such litigation is filed.
-
-   4. Redistribution. You may reproduce and distribute copies of the
-      Work or Derivative Works thereof in any medium, with or without
-      modifications, and in Source or Object form, provided that You
-      meet the following conditions:
-
-      (a) You must give any other recipients of the Work or
-          Derivative Works a copy of this License; and
-
-      (b) You must cause any modified files to carry prominent notices
-          stating that You changed the files; and
-
-      (c) You must retain, in the Source form of any Derivative Works
-          that You distribute, all copyright, patent, trademark, and
-          attribution notices from the Source form of the Work,
-          excluding those notices that do not pertain to any part of
-          the Derivative Works; and
-
-      (d) If the Work includes a "NOTICE" text file as part of its
-          distribution, then any Derivative Works that You distribute must
-          include a readable copy of the attribution notices contained
-          within such NOTICE file, excluding those notices that do not
-          pertain to any part of the Derivative Works, in at least one
-          of the following places: within a NOTICE text file distributed
-          as part of the Derivative Works; within the Source form or
-          documentation, if provided along with the Derivative Works; or,
-          within a display generated by the Derivative Works, if and
-          wherever such third-party notices normally appear. The contents
-          of the NOTICE file are for informational purposes only and
-          do not modify the License. You may add Your own attribution
-          notices within Derivative Works that You distribute, alongside
-          or as an addendum to the NOTICE text from the Work, provided
-          that such additional attribution notices cannot be construed
-          as modifying the License.
-
-      You may add Your own copyright statement to Your modifications and
-      may provide additional or different license terms and conditions
-      for use, reproduction, or distribution of Your modifications, or
-      for any such Derivative Works as a whole, provided Your use,
-      reproduction, and distribution of the Work otherwise complies with
-      the conditions stated in this License.
-
-   5. Submission of Contributions. Unless You explicitly state otherwise,
-      any Contribution intentionally submitted for inclusion in the Work
-      by You to the Licensor shall be under the terms and conditions of
-      this License, without any additional terms or conditions.
-      Notwithstanding the above, nothing herein shall supersede or modify
-      the terms of any separate license agreement you may have executed
-      with Licensor regarding such Contributions.
-
-   6. Trademarks. This License does not grant permission to use the trade
-      names, trademarks, service marks, or product names of the Licensor,
-      except as required for reasonable and customary use in describing the
-      origin of the Work and reproducing the content of the NOTICE file.
-
-   7. Disclaimer of Warranty. Unless required by applicable law or
-      agreed to in writing, Licensor provides the Work (and each
-      Contributor provides its Contributions) on an "AS IS" BASIS,
-      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
-      implied, including, without limitation, any warranties or conditions
-      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
-      PARTICULAR PURPOSE. You are solely responsible for determining the
-      appropriateness of using or redistributing the Work and assume any
-      risks associated with Your exercise of permissions under this License.
-
-   8. Limitation of Liability. In no event and under no legal theory,
-      whether in tort (including negligence), contract, or otherwise,
-      unless required by applicable law (such as deliberate and grossly
-      negligent acts) or agreed to in writing, shall any Contributor be
-      liable to You for damages, including any direct, indirect, special,
-      incidental, or consequential damages of any character arising as a
-      result of this License or out of the use or inability to use the
-      Work (including but not limited to damages for loss of goodwill,
-      work stoppage, computer failure or malfunction, or any and all
-      other commercial damages or losses), even if such Contributor
-      has been advised of the possibility of such damages.
-
-   9. Accepting Warranty or Additional Liability. While redistributing
-      the Work or Derivative Works thereof, You may choose to offer,
-      and charge a fee for, acceptance of support, warranty, indemnity,
-      or other liability obligations and/or rights consistent with this
-      License. However, in accepting such obligations, You may act only
-      on Your own behalf and on Your sole responsibility, not on behalf
-      of any other Contributor, and only if You agree to indemnify,
-      defend, and hold each Contributor harmless for any liability
-      incurred by, or claims asserted against, such Contributor by reason
-      of your accepting any such warranty or additional liability.
-
-   END OF TERMS AND CONDITIONS
-
-   APPENDIX: How to apply the Apache License to your work.
-
-      To apply the Apache License to your work, attach the following
-      boilerplate notice, with the fields enclosed by brackets "[]"
-      replaced with your own identifying information. (Don't include
-      the brackets!)  The text should be enclosed in the appropriate
-      comment syntax for the file format. We also recommend that a
-      file or class name and description of purpose be included on the
-      same "printed page" as the copyright notice for easier
-      identification within third-party archives.
-
-   Copyright [yyyy] [name of copyright owner]
-
-   Licensed 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.


[19/50] [abbrv] beam git commit: Install test dependencies in the post commit script.

Posted by da...@apache.org.
Install test dependencies in the post commit script.


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

Branch: refs/heads/master
Commit: 19789db9cab031e0891cb67c4ab6b8b03c6a8c09
Parents: 43cb4d7
Author: Ahmet Altay <al...@google.com>
Authored: Tue Jan 24 15:13:03 2017 -0800
Committer: Robert Bradshaw <ro...@gmail.com>
Committed: Tue Jan 24 16:32:36 2017 -0800

----------------------------------------------------------------------
 sdks/python/run_postcommit.sh | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/19789db9/sdks/python/run_postcommit.sh
----------------------------------------------------------------------
diff --git a/sdks/python/run_postcommit.sh b/sdks/python/run_postcommit.sh
index 2e419a5..3756075 100755
--- a/sdks/python/run_postcommit.sh
+++ b/sdks/python/run_postcommit.sh
@@ -70,6 +70,10 @@ python setup.py sdist
 
 SDK_LOCATION=$(find dist/apache-beam-sdk-*.tar.gz)
 
+# Install test dependencies for ValidatesRunner tests.
+echo "pyhamcrest" > postcommit_requirements.txt
+echo "mock" >> postcommit_requirements.txt
+
 # Run ValidatesRunner tests on Google Cloud Dataflow service
 echo ">>> RUNNING DATAFLOW RUNNER VALIDATESRUNNER TESTS"
 python setup.py nosetests \
@@ -80,6 +84,7 @@ python setup.py nosetests \
     --temp_location=$GCS_LOCATION/temp-validatesrunner-test \
     --sdk_location=$SDK_LOCATION \
     --job_name=$JOBNAME_VR_TEST \
+    --requirements_file=postcommit_requirements.txt \
     --num_workers=1"
 
 # Run wordcount on the Google Cloud Dataflow service


[10/50] [abbrv] beam git commit: Remove dataflow_test.py

Posted by da...@apache.org.
Remove dataflow_test.py

Many of these tests were redundant with tests elsewhere, and the ones
that weren't were put closer to similar tests.


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

Branch: refs/heads/master
Commit: d5b90d8383e662e803ea79b31661250a043bcfd2
Parents: 01b3628
Author: Robert Bradshaw <ro...@gmail.com>
Authored: Sat Jan 21 21:53:42 2017 -0800
Committer: Robert Bradshaw <ro...@google.com>
Committed: Mon Jan 23 14:36:55 2017 -0800

----------------------------------------------------------------------
 sdks/python/apache_beam/dataflow_test.py        | 418 -------------------
 .../apache_beam/transforms/ptransform_test.py   |  67 +++
 .../apache_beam/transforms/sideinputs_test.py   | 208 ++++++++-
 3 files changed, 274 insertions(+), 419 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/d5b90d83/sdks/python/apache_beam/dataflow_test.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/dataflow_test.py b/sdks/python/apache_beam/dataflow_test.py
deleted file mode 100644
index f410230..0000000
--- a/sdks/python/apache_beam/dataflow_test.py
+++ /dev/null
@@ -1,418 +0,0 @@
-#
-# 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.
-#
-
-"""Integration tests for the dataflow package."""
-
-from __future__ import absolute_import
-
-import logging
-import re
-import unittest
-
-import apache_beam as beam
-from apache_beam.pvalue import AsDict
-from apache_beam.pvalue import AsIter as AllOf
-from apache_beam.pvalue import AsList
-from apache_beam.pvalue import AsSingleton
-from apache_beam.pvalue import EmptySideInput
-from apache_beam.pvalue import SideOutputValue
-from apache_beam.test_pipeline import TestPipeline
-from apache_beam.transforms import Create
-from apache_beam.transforms import DoFn
-from apache_beam.transforms import FlatMap
-from apache_beam.transforms import GroupByKey
-from apache_beam.transforms import Map
-from apache_beam.transforms import ParDo
-from apache_beam.transforms import WindowInto
-from apache_beam.transforms.util import assert_that
-from apache_beam.transforms.util import equal_to
-from apache_beam.transforms.window import IntervalWindow
-from apache_beam.transforms.window import WindowFn
-from nose.plugins.attrib import attr
-
-
-class DataflowTest(unittest.TestCase):
-  """Dataflow integration tests."""
-
-  SAMPLE_DATA = ['aa bb cc aa bb aa \n'] * 10
-  SAMPLE_RESULT = [('cc', 10), ('bb', 20), ('aa', 30)]
-
-  @beam.ptransform_fn
-  def Count(pcoll):  # pylint: disable=invalid-name, no-self-argument
-    """A Count transform: v, ... => (v, n), ..."""
-    return (pcoll
-            | 'AddCount' >> Map(lambda x: (x, 1))
-            | 'GroupCounts' >> GroupByKey()
-            | 'AddCounts' >> Map(lambda (x, ones): (x, sum(ones))))
-
-  @attr('ValidatesRunner')
-  def test_word_count(self):
-    pipeline = TestPipeline()
-    lines = pipeline | 'SomeWords' >> Create(DataflowTest.SAMPLE_DATA)
-    result = (
-        (lines | 'GetWords' >> FlatMap(lambda x: re.findall(r'\w+', x)))
-        .apply('CountWords', DataflowTest.Count))
-    assert_that(result, equal_to(DataflowTest.SAMPLE_RESULT))
-    pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_map(self):
-    pipeline = TestPipeline()
-    lines = pipeline | 'input' >> Create(['a', 'b', 'c'])
-    result = (lines
-              | 'upper' >> Map(str.upper)
-              | 'prefix' >> Map(lambda x, prefix: prefix + x, 'foo-'))
-    assert_that(result, equal_to(['foo-A', 'foo-B', 'foo-C']))
-    pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_par_do_with_side_input_as_arg(self):
-    pipeline = TestPipeline()
-    words_list = ['aa', 'bb', 'cc']
-    words = pipeline | 'SomeWords' >> Create(words_list)
-    prefix = pipeline | 'SomeString' >> Create(['xyz'])  # side in
-    suffix = 'zyx'
-    result = words | FlatMap(
-        'DecorateWords',
-        lambda x, pfx, sfx: ['%s-%s-%s' % (pfx, x, sfx)],
-        AsSingleton(prefix), suffix)
-    assert_that(result, equal_to(['xyz-%s-zyx' % x for x in words_list]))
-    pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_par_do_with_side_input_as_keyword_arg(self):
-    pipeline = TestPipeline()
-    words_list = ['aa', 'bb', 'cc']
-    words = pipeline | 'SomeWords' >> Create(words_list)
-    prefix = 'zyx'
-    suffix = pipeline | 'SomeString' >> Create(['xyz'])  # side in
-    result = words | FlatMap(
-        'DecorateWords',
-        lambda x, pfx, sfx: ['%s-%s-%s' % (pfx, x, sfx)],
-        prefix, sfx=AsSingleton(suffix))
-    assert_that(result, equal_to(['zyx-%s-xyz' % x for x in words_list]))
-    pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_par_do_with_do_fn_object(self):
-    class SomeDoFn(DoFn):
-      """A custom DoFn for a FlatMap transform."""
-
-      def process(self, context, prefix, suffix):
-        return ['%s-%s-%s' % (prefix, context.element, suffix)]
-
-    pipeline = TestPipeline()
-    words_list = ['aa', 'bb', 'cc']
-    words = pipeline | 'SomeWords' >> Create(words_list)
-    prefix = 'zyx'
-    suffix = pipeline | 'SomeString' >> Create(['xyz'])  # side in
-    result = words | 'DecorateWordsDoFn' >> ParDo(
-        SomeDoFn(), prefix, suffix=AsSingleton(suffix))
-    assert_that(result, equal_to(['zyx-%s-xyz' % x for x in words_list]))
-    pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_par_do_with_multiple_outputs_and_using_yield(self):
-    class SomeDoFn(DoFn):
-      """A custom DoFn using yield."""
-
-      def process(self, context):
-        yield context.element
-        if context.element % 2 == 0:
-          yield SideOutputValue('even', context.element)
-        else:
-          yield SideOutputValue('odd', context.element)
-
-    pipeline = TestPipeline()
-    nums = pipeline | 'Some Numbers' >> Create([1, 2, 3, 4])
-    results = nums | ParDo(
-        'ClassifyNumbers', SomeDoFn()).with_outputs('odd', 'even', main='main')
-    assert_that(results.main, equal_to([1, 2, 3, 4]))
-    assert_that(results.odd, equal_to([1, 3]), label='assert:odd')
-    assert_that(results.even, equal_to([2, 4]), label='assert:even')
-    pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_par_do_with_multiple_outputs_and_using_return(self):
-    def some_fn(v):
-      if v % 2 == 0:
-        return [v, SideOutputValue('even', v)]
-      else:
-        return [v, SideOutputValue('odd', v)]
-
-    pipeline = TestPipeline()
-    nums = pipeline | 'Some Numbers' >> Create([1, 2, 3, 4])
-    results = nums | FlatMap(
-        'ClassifyNumbers', some_fn).with_outputs('odd', 'even', main='main')
-    assert_that(results.main, equal_to([1, 2, 3, 4]))
-    assert_that(results.odd, equal_to([1, 3]), label='assert:odd')
-    assert_that(results.even, equal_to([2, 4]), label='assert:even')
-    pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_empty_singleton_side_input(self):
-    pipeline = TestPipeline()
-    pcol = pipeline | 'start' >> Create([1, 2])
-    side = pipeline | 'side' >> Create([])  # Empty side input.
-
-    def my_fn(k, s):
-      v = ('empty' if isinstance(s, EmptySideInput) else 'full')
-      return [(k, v)]
-    result = pcol | 'compute' >> FlatMap(my_fn, AsSingleton(side))
-    assert_that(result, equal_to([(1, 'empty'), (2, 'empty')]))
-    pipeline.run()
-
-  # @attr('ValidatesRunner')
-  # TODO(BEAM-1124): Temporarily disable it due to test failed running on
-  # Dataflow service.
-  def test_multi_valued_singleton_side_input(self):
-    pipeline = TestPipeline()
-    pcol = pipeline | 'start' >> Create([1, 2])
-    side = pipeline | 'side' >> Create([3, 4])  # 2 values in side input.
-    pcol | 'compute' >> FlatMap(lambda x, s: [x * s], AsSingleton(side))  # pylint: disable=expression-not-assigned
-    with self.assertRaises(ValueError):
-      pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_default_value_singleton_side_input(self):
-    pipeline = TestPipeline()
-    pcol = pipeline | 'start' >> Create([1, 2])
-    side = pipeline | 'side' >> Create([])  # 0 values in side input.
-    result = pcol | FlatMap(lambda x, s: [x * s], AsSingleton(side, 10))
-    assert_that(result, equal_to([10, 20]))
-    pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_iterable_side_input(self):
-    pipeline = TestPipeline()
-    pcol = pipeline | 'start' >> Create([1, 2])
-    side = pipeline | 'side' >> Create([3, 4])  # 2 values in side input.
-    result = pcol | FlatMap('compute',
-                            lambda x, s: [x * y for y in s], AllOf(side))
-    assert_that(result, equal_to([3, 4, 6, 8]))
-    pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_undeclared_side_outputs(self):
-    pipeline = TestPipeline()
-    nums = pipeline | 'Some Numbers' >> Create([1, 2, 3, 4])
-    results = nums | FlatMap(
-        'ClassifyNumbers',
-        lambda x: [x, SideOutputValue('even' if x % 2 == 0 else 'odd', x)]
-    ).with_outputs()
-    assert_that(results[None], equal_to([1, 2, 3, 4]))
-    assert_that(results.odd, equal_to([1, 3]), label='assert:odd')
-    assert_that(results.even, equal_to([2, 4]), label='assert:even')
-    pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_empty_side_outputs(self):
-    pipeline = TestPipeline()
-    nums = pipeline | 'Some Numbers' >> Create([1, 3, 5])
-    results = nums | FlatMap(
-        'ClassifyNumbers',
-        lambda x: [x, SideOutputValue('even' if x % 2 == 0 else 'odd', x)]
-    ).with_outputs()
-    assert_that(results[None], equal_to([1, 3, 5]))
-    assert_that(results.odd, equal_to([1, 3, 5]), label='assert:odd')
-    assert_that(results.even, equal_to([]), label='assert:even')
-    pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_as_list_and_as_dict_side_inputs(self):
-    a_list = [5, 1, 3, 2, 9]
-    some_pairs = [('crouton', 17), ('supreme', None)]
-    pipeline = TestPipeline()
-    main_input = pipeline | 'main input' >> Create([1])
-    side_list = pipeline | 'side list' >> Create(a_list)
-    side_pairs = pipeline | 'side pairs' >> Create(some_pairs)
-    results = main_input | FlatMap(
-        'concatenate',
-        lambda x, the_list, the_dict: [[x, the_list, the_dict]],
-        AsList(side_list), AsDict(side_pairs))
-
-    def  matcher(expected_elem, expected_list, expected_pairs):
-      def match(actual):
-        [[actual_elem, actual_list, actual_dict]] = actual
-        equal_to([expected_elem])([actual_elem])
-        equal_to(expected_list)(actual_list)
-        equal_to(expected_pairs)(actual_dict.iteritems())
-      return match
-
-    assert_that(results, matcher(1, a_list, some_pairs))
-    pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_as_singleton_without_unique_labels(self):
-    # This should succeed as calling AsSingleton on the same PCollection twice
-    # with the same defaults will return the same PCollectionView.
-    a_list = [2]
-    pipeline = TestPipeline()
-    main_input = pipeline | 'main input' >> Create([1])
-    side_list = pipeline | 'side list' >> Create(a_list)
-    results = main_input | FlatMap(
-        'test',
-        lambda x, s1, s2: [[x, s1, s2]],
-        AsSingleton(side_list), AsSingleton(side_list))
-
-    def  matcher(expected_elem, expected_singleton):
-      def match(actual):
-        [[actual_elem, actual_singleton1, actual_singleton2]] = actual
-        equal_to([expected_elem])([actual_elem])
-        equal_to([expected_singleton])([actual_singleton1])
-        equal_to([expected_singleton])([actual_singleton2])
-      return match
-
-    assert_that(results, matcher(1, 2))
-    pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_as_singleton_with_different_defaults_without_unique_labels(self):
-    # This should fail as AsSingleton with distinct default values should create
-    # distinct PCollectionViews with the same full_label.
-    a_list = [2]
-    pipeline = TestPipeline()
-    main_input = pipeline | 'main input' >> Create([1])
-    side_list = pipeline | 'side list' >> Create(a_list)
-
-    with self.assertRaises(RuntimeError) as e:
-      _ = main_input | FlatMap(
-          'test',
-          lambda x, s1, s2: [[x, s1, s2]],
-          AsSingleton(side_list), AsSingleton(side_list, default_value=3))
-    self.assertTrue(
-        e.exception.message.startswith(
-            'Transform "ViewAsSingleton(side list.None)" does not have a '
-            'stable unique label.'))
-
-  @attr('ValidatesRunner')
-  def test_as_singleton_with_different_defaults_with_unique_labels(self):
-    a_list = []
-    pipeline = TestPipeline()
-    main_input = pipeline | 'main input' >> Create([1])
-    side_list = pipeline | 'side list' >> Create(a_list)
-    results = main_input | FlatMap(
-        'test',
-        lambda x, s1, s2: [[x, s1, s2]],
-        AsSingleton('si1', side_list, default_value=2),
-        AsSingleton('si2', side_list, default_value=3))
-
-    def  matcher(expected_elem, expected_singleton1, expected_singleton2):
-      def match(actual):
-        [[actual_elem, actual_singleton1, actual_singleton2]] = actual
-        equal_to([expected_elem])([actual_elem])
-        equal_to([expected_singleton1])([actual_singleton1])
-        equal_to([expected_singleton2])([actual_singleton2])
-      return match
-
-    assert_that(results, matcher(1, 2, 3))
-    pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_as_list_without_unique_labels(self):
-    # This should succeed as calling AsList on the same PCollection twice will
-    # return the same PCollectionView.
-    a_list = [1, 2, 3]
-    pipeline = TestPipeline()
-    main_input = pipeline | 'main input' >> Create([1])
-    side_list = pipeline | 'side list' >> Create(a_list)
-    results = main_input | FlatMap(
-        'test',
-        lambda x, ls1, ls2: [[x, ls1, ls2]],
-        AsList(side_list), AsList(side_list))
-
-    def  matcher(expected_elem, expected_list):
-      def match(actual):
-        [[actual_elem, actual_list1, actual_list2]] = actual
-        equal_to([expected_elem])([actual_elem])
-        equal_to(expected_list)(actual_list1)
-        equal_to(expected_list)(actual_list2)
-      return match
-
-    assert_that(results, matcher(1, [1, 2, 3]))
-    pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_as_list_with_unique_labels(self):
-    a_list = [1, 2, 3]
-    pipeline = TestPipeline()
-    main_input = pipeline | 'main input' >> Create([1])
-    side_list = pipeline | 'side list' >> Create(a_list)
-    results = main_input | FlatMap(
-        'test',
-        lambda x, ls1, ls2: [[x, ls1, ls2]],
-        AsList(side_list), AsList(side_list, label='label'))
-
-    def  matcher(expected_elem, expected_list):
-      def match(actual):
-        [[actual_elem, actual_list1, actual_list2]] = actual
-        equal_to([expected_elem])([actual_elem])
-        equal_to(expected_list)(actual_list1)
-        equal_to(expected_list)(actual_list2)
-      return match
-
-    assert_that(results, matcher(1, [1, 2, 3]))
-    pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_as_dict_with_unique_labels(self):
-    some_kvs = [('a', 1), ('b', 2)]
-    pipeline = TestPipeline()
-    main_input = pipeline | 'main input' >> Create([1])
-    side_kvs = pipeline | 'side kvs' >> Create(some_kvs)
-    results = main_input | FlatMap(
-        'test',
-        lambda x, dct1, dct2: [[x, dct1, dct2]],
-        AsDict(side_kvs), AsDict(side_kvs, label='label'))
-
-    def  matcher(expected_elem, expected_kvs):
-      def match(actual):
-        [[actual_elem, actual_dict1, actual_dict2]] = actual
-        equal_to([expected_elem])([actual_elem])
-        equal_to(expected_kvs)(actual_dict1.iteritems())
-        equal_to(expected_kvs)(actual_dict2.iteritems())
-      return match
-
-    assert_that(results, matcher(1, some_kvs))
-    pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_window_transform(self):
-    class TestWindowFn(WindowFn):
-      """Windowing function adding two disjoint windows to each element."""
-
-      def assign(self, assign_context):
-        _ = assign_context
-        return [IntervalWindow(10, 20), IntervalWindow(20, 30)]
-
-      def merge(self, existing_windows):
-        return existing_windows
-
-    pipeline = TestPipeline()
-    numbers = pipeline | 'KVs' >> Create([(1, 10), (2, 20), (3, 30)])
-    result = (numbers
-              | 'W' >> WindowInto(windowfn=TestWindowFn())
-              | 'G' >> GroupByKey())
-    assert_that(
-        result, equal_to([(1, [10]), (1, [10]), (2, [20]),
-                          (2, [20]), (3, [30]), (3, [30])]))
-    pipeline.run()
-
-
-if __name__ == '__main__':
-  logging.getLogger().setLevel(logging.INFO)
-  unittest.main()

http://git-wip-us.apache.org/repos/asf/beam/blob/d5b90d83/sdks/python/apache_beam/transforms/ptransform_test.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/transforms/ptransform_test.py b/sdks/python/apache_beam/transforms/ptransform_test.py
index 827bc83..68e4482 100644
--- a/sdks/python/apache_beam/transforms/ptransform_test.py
+++ b/sdks/python/apache_beam/transforms/ptransform_test.py
@@ -24,6 +24,7 @@ import re
 import unittest
 
 import hamcrest as hc
+from nose.plugins.attrib import attr
 
 import apache_beam as beam
 from apache_beam.test_pipeline import TestPipeline
@@ -189,6 +190,72 @@ class PTransformTest(unittest.TestCase):
     assert_that(r2.m, equal_to([3, 4, 5]), label='r2')
     pipeline.run()
 
+  @attr('ValidatesRunner')
+  def test_par_do_with_multiple_outputs_and_using_yield(self):
+    class SomeDoFn(beam.DoFn):
+      """A custom DoFn using yield."""
+
+      def process(self, context):
+        yield context.element
+        if context.element % 2 == 0:
+          yield pvalue.SideOutputValue('even', context.element)
+        else:
+          yield pvalue.SideOutputValue('odd', context.element)
+
+    pipeline = TestPipeline()
+    nums = pipeline | 'Some Numbers' >> beam.Create([1, 2, 3, 4])
+    results = nums | beam.ParDo(
+        'ClassifyNumbers', SomeDoFn()).with_outputs('odd', 'even', main='main')
+    assert_that(results.main, equal_to([1, 2, 3, 4]))
+    assert_that(results.odd, equal_to([1, 3]), label='assert:odd')
+    assert_that(results.even, equal_to([2, 4]), label='assert:even')
+    pipeline.run()
+
+  @attr('ValidatesRunner')
+  def test_par_do_with_multiple_outputs_and_using_return(self):
+    def some_fn(v):
+      if v % 2 == 0:
+        return [v, pvalue.SideOutputValue('even', v)]
+      else:
+        return [v, pvalue.SideOutputValue('odd', v)]
+
+    pipeline = TestPipeline()
+    nums = pipeline | 'Some Numbers' >> beam.Create([1, 2, 3, 4])
+    results = nums | beam.FlatMap(
+        'ClassifyNumbers', some_fn).with_outputs('odd', 'even', main='main')
+    assert_that(results.main, equal_to([1, 2, 3, 4]))
+    assert_that(results.odd, equal_to([1, 3]), label='assert:odd')
+    assert_that(results.even, equal_to([2, 4]), label='assert:even')
+    pipeline.run()
+
+  @attr('ValidatesRunner')
+  def test_undeclared_side_outputs(self):
+    pipeline = TestPipeline()
+    nums = pipeline | 'Some Numbers' >> beam.Create([1, 2, 3, 4])
+    results = nums | beam.FlatMap(
+        'ClassifyNumbers',
+        lambda x: [x,
+                   pvalue.SideOutputValue('even' if x % 2 == 0 else 'odd', x)]
+    ).with_outputs()
+    assert_that(results[None], equal_to([1, 2, 3, 4]))
+    assert_that(results.odd, equal_to([1, 3]), label='assert:odd')
+    assert_that(results.even, equal_to([2, 4]), label='assert:even')
+    pipeline.run()
+
+  @attr('ValidatesRunner')
+  def test_empty_side_outputs(self):
+    pipeline = TestPipeline()
+    nums = pipeline | 'Some Numbers' >> beam.Create([1, 3, 5])
+    results = nums | beam.FlatMap(
+        'ClassifyNumbers',
+        lambda x: [x,
+                   pvalue.SideOutputValue('even' if x % 2 == 0 else 'odd', x)]
+    ).with_outputs()
+    assert_that(results[None], equal_to([1, 3, 5]))
+    assert_that(results.odd, equal_to([1, 3, 5]), label='assert:odd')
+    assert_that(results.even, equal_to([]), label='assert:even')
+    pipeline.run()
+
   def test_do_requires_do_fn_returning_iterable(self):
     # This function is incorrect because it returns an object that isn't an
     # iterable.

http://git-wip-us.apache.org/repos/asf/beam/blob/d5b90d83/sdks/python/apache_beam/transforms/sideinputs_test.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/transforms/sideinputs_test.py b/sdks/python/apache_beam/transforms/sideinputs_test.py
index a327dc8..9384e7b 100644
--- a/sdks/python/apache_beam/transforms/sideinputs_test.py
+++ b/sdks/python/apache_beam/transforms/sideinputs_test.py
@@ -20,7 +20,10 @@
 import logging
 import unittest
 
+from nose.plugins.attrib import attr
+
 import apache_beam as beam
+from apache_beam.test_pipeline import TestPipeline
 from apache_beam.transforms import window
 from apache_beam.transforms.util import assert_that, equal_to
 
@@ -28,7 +31,7 @@ from apache_beam.transforms.util import assert_that, equal_to
 class SideInputsTest(unittest.TestCase):
 
   def create_pipeline(self):
-    return beam.Pipeline('DirectRunner')
+    return TestPipeline('DirectRunner')
 
   def run_windowed_side_inputs(self, elements, main_window_fn,
                                side_window_fn=None,
@@ -125,6 +128,209 @@ class SideInputsTest(unittest.TestCase):
             (11, {'k11': 'v11'}),
         ])
 
+  @attr('ValidatesRunner')
+  def test_empty_singleton_side_input(self):
+    pipeline = self.create_pipeline()
+    pcol = pipeline | 'start' >> beam.Create([1, 2])
+    side = pipeline | 'side' >> beam.Create([])  # Empty side input.
+
+    def my_fn(k, s):
+      # TODO(robertwb): Should this be an error as in Java?
+      v = ('empty' if isinstance(s, beam.pvalue.EmptySideInput) else 'full')
+      return [(k, v)]
+    result = pcol | 'compute' >> beam.FlatMap(
+        my_fn, beam.pvalue.AsSingleton(side))
+    assert_that(result, equal_to([(1, 'empty'), (2, 'empty')]))
+    pipeline.run()
+
+  # @attr('ValidatesRunner')
+  # TODO(BEAM-1124): Temporarily disable it due to test failed running on
+  # Dataflow service.
+  def test_multi_valued_singleton_side_input(self):
+    pipeline = self.create_pipeline()
+    pcol = pipeline | 'start' >> beam.Create([1, 2])
+    side = pipeline | 'side' >> beam.Create([3, 4])  # 2 values in side input.
+    pcol | 'compute' >> beam.FlatMap(  # pylint: disable=expression-not-assigned
+        lambda x, s: [x * s], beam.pvalue.AsSingleton(side))
+    with self.assertRaises(ValueError):
+      pipeline.run()
+
+  @attr('ValidatesRunner')
+  def test_default_value_singleton_side_input(self):
+    pipeline = self.create_pipeline()
+    pcol = pipeline | 'start' >> beam.Create([1, 2])
+    side = pipeline | 'side' >> beam.Create([])  # 0 values in side input.
+    result = pcol | beam.FlatMap(
+        lambda x, s: [x * s], beam.pvalue.AsSingleton(side, 10))
+    assert_that(result, equal_to([10, 20]))
+    pipeline.run()
+
+  @attr('ValidatesRunner')
+  def test_iterable_side_input(self):
+    pipeline = self.create_pipeline()
+    pcol = pipeline | 'start' >> beam.Create([1, 2])
+    side = pipeline | 'side' >> beam.Create([3, 4])  # 2 values in side input.
+    result = pcol | 'compute' >> beam.FlatMap(
+        lambda x, s: [x * y for y in s],
+        beam.pvalue.AsIter(side))
+    assert_that(result, equal_to([3, 4, 6, 8]))
+    pipeline.run()
+
+  @attr('ValidatesRunner')
+  def test_as_list_and_as_dict_side_inputs(self):
+    a_list = [5, 1, 3, 2, 9]
+    some_pairs = [('crouton', 17), ('supreme', None)]
+    pipeline = self.create_pipeline()
+    main_input = pipeline | 'main input' >> beam.Create([1])
+    side_list = pipeline | 'side list' >> beam.Create(a_list)
+    side_pairs = pipeline | 'side pairs' >> beam.Create(some_pairs)
+    results = main_input | 'concatenate' >> beam.FlatMap(
+        lambda x, the_list, the_dict: [[x, the_list, the_dict]],
+        beam.pvalue.AsList(side_list), beam.pvalue.AsDict(side_pairs))
+
+    def  matcher(expected_elem, expected_list, expected_pairs):
+      def match(actual):
+        [[actual_elem, actual_list, actual_dict]] = actual
+        equal_to([expected_elem])([actual_elem])
+        equal_to(expected_list)(actual_list)
+        equal_to(expected_pairs)(actual_dict.iteritems())
+      return match
+
+    assert_that(results, matcher(1, a_list, some_pairs))
+    pipeline.run()
+
+  @attr('ValidatesRunner')
+  def test_as_singleton_without_unique_labels(self):
+    # This should succeed as calling beam.pvalue.AsSingleton on the same
+    # PCollection twice with the same defaults will return the same
+    # PCollectionView.
+    a_list = [2]
+    pipeline = self.create_pipeline()
+    main_input = pipeline | 'main input' >> beam.Create([1])
+    side_list = pipeline | 'side list' >> beam.Create(a_list)
+    results = main_input | beam.FlatMap(
+        lambda x, s1, s2: [[x, s1, s2]],
+        beam.pvalue.AsSingleton(side_list), beam.pvalue.AsSingleton(side_list))
+
+    def  matcher(expected_elem, expected_singleton):
+      def match(actual):
+        [[actual_elem, actual_singleton1, actual_singleton2]] = actual
+        equal_to([expected_elem])([actual_elem])
+        equal_to([expected_singleton])([actual_singleton1])
+        equal_to([expected_singleton])([actual_singleton2])
+      return match
+
+    assert_that(results, matcher(1, 2))
+    pipeline.run()
+
+  @attr('ValidatesRunner')
+  def test_as_singleton_with_different_defaults_without_unique_labels(self):
+    # This should fail as beam.pvalue.AsSingleton with distinct default values
+    # should beam.Create distinct PCollectionViews with the same full_label.
+    a_list = [2]
+    pipeline = self.create_pipeline()
+    main_input = pipeline | 'main input' >> beam.Create([1])
+    side_list = pipeline | 'side list' >> beam.Create(a_list)
+
+    with self.assertRaises(RuntimeError) as e:
+      _ = main_input | beam.FlatMap(
+          lambda x, s1, s2: [[x, s1, s2]],
+          beam.pvalue.AsSingleton(side_list),
+          beam.pvalue.AsSingleton(side_list, default_value=3))
+    self.assertTrue(
+        e.exception.message.startswith(
+            'Transform "ViewAsSingleton(side list.None)" does not have a '
+            'stable unique label.'))
+
+  @attr('ValidatesRunner')
+  def test_as_singleton_with_different_defaults_with_unique_labels(self):
+    a_list = []
+    pipeline = self.create_pipeline()
+    main_input = pipeline | 'main input' >> beam.Create([1])
+    side_list = pipeline | 'side list' >> beam.Create(a_list)
+    results = main_input | beam.FlatMap(
+        lambda x, s1, s2: [[x, s1, s2]],
+        beam.pvalue.AsSingleton('si1', side_list, default_value=2),
+        beam.pvalue.AsSingleton('si2', side_list, default_value=3))
+
+    def  matcher(expected_elem, expected_singleton1, expected_singleton2):
+      def match(actual):
+        [[actual_elem, actual_singleton1, actual_singleton2]] = actual
+        equal_to([expected_elem])([actual_elem])
+        equal_to([expected_singleton1])([actual_singleton1])
+        equal_to([expected_singleton2])([actual_singleton2])
+      return match
+
+    assert_that(results, matcher(1, 2, 3))
+    pipeline.run()
+
+  @attr('ValidatesRunner')
+  def test_as_list_without_unique_labels(self):
+    # This should succeed as calling beam.pvalue.AsList on the same
+    # PCollection twice will return the same PCollectionView.
+    a_list = [1, 2, 3]
+    pipeline = self.create_pipeline()
+    main_input = pipeline | 'main input' >> beam.Create([1])
+    side_list = pipeline | 'side list' >> beam.Create(a_list)
+    results = main_input | beam.FlatMap(
+        lambda x, ls1, ls2: [[x, ls1, ls2]],
+        beam.pvalue.AsList(side_list), beam.pvalue.AsList(side_list))
+
+    def  matcher(expected_elem, expected_list):
+      def match(actual):
+        [[actual_elem, actual_list1, actual_list2]] = actual
+        equal_to([expected_elem])([actual_elem])
+        equal_to(expected_list)(actual_list1)
+        equal_to(expected_list)(actual_list2)
+      return match
+
+    assert_that(results, matcher(1, [1, 2, 3]))
+    pipeline.run()
+
+  @attr('ValidatesRunner')
+  def test_as_list_with_unique_labels(self):
+    a_list = [1, 2, 3]
+    pipeline = self.create_pipeline()
+    main_input = pipeline | 'main input' >> beam.Create([1])
+    side_list = pipeline | 'side list' >> beam.Create(a_list)
+    results = main_input | beam.FlatMap(
+        lambda x, ls1, ls2: [[x, ls1, ls2]],
+        beam.pvalue.AsList(side_list),
+        beam.pvalue.AsList(side_list, label='label'))
+
+    def  matcher(expected_elem, expected_list):
+      def match(actual):
+        [[actual_elem, actual_list1, actual_list2]] = actual
+        equal_to([expected_elem])([actual_elem])
+        equal_to(expected_list)(actual_list1)
+        equal_to(expected_list)(actual_list2)
+      return match
+
+    assert_that(results, matcher(1, [1, 2, 3]))
+    pipeline.run()
+
+  @attr('ValidatesRunner')
+  def test_as_dict_with_unique_labels(self):
+    some_kvs = [('a', 1), ('b', 2)]
+    pipeline = self.create_pipeline()
+    main_input = pipeline | 'main input' >> beam.Create([1])
+    side_kvs = pipeline | 'side kvs' >> beam.Create(some_kvs)
+    results = main_input | beam.FlatMap(
+        lambda x, dct1, dct2: [[x, dct1, dct2]],
+        beam.pvalue.AsDict(side_kvs),
+        beam.pvalue.AsDict(side_kvs, label='label'))
+
+    def  matcher(expected_elem, expected_kvs):
+      def match(actual):
+        [[actual_elem, actual_dict1, actual_dict2]] = actual
+        equal_to([expected_elem])([actual_elem])
+        equal_to(expected_kvs)(actual_dict1.iteritems())
+        equal_to(expected_kvs)(actual_dict2.iteritems())
+      return match
+
+    assert_that(results, matcher(1, some_kvs))
+    pipeline.run()
+
 
 if __name__ == '__main__':
   logging.getLogger().setLevel(logging.DEBUG)


[18/50] [abbrv] beam git commit: Closes #1820

Posted by da...@apache.org.
Closes #1820


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

Branch: refs/heads/master
Commit: 43cb4d70980af758bfea9a3c65530ca53a6239ec
Parents: f983123 52fc95d
Author: Robert Bradshaw <ro...@gmail.com>
Authored: Tue Jan 24 16:31:03 2017 -0800
Committer: Robert Bradshaw <ro...@gmail.com>
Committed: Tue Jan 24 16:31:03 2017 -0800

----------------------------------------------------------------------
 sdks/python/apache_beam/io/fileio.py            | 542 +-------------
 sdks/python/apache_beam/io/fileio_test.py       | 729 +------------------
 .../runners/direct/transform_evaluator.py       |   5 -
 3 files changed, 3 insertions(+), 1273 deletions(-)
----------------------------------------------------------------------



[37/50] [abbrv] beam git commit: Update pom.xml for sdks/python.

Posted by da...@apache.org.
Update pom.xml for sdks/python.


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

Branch: refs/heads/master
Commit: f1b8679c4af283d1e751043e2e765b7f295af0b2
Parents: c2859a5
Author: Ahmet Altay <al...@google.com>
Authored: Fri Jan 27 17:04:21 2017 -0800
Committer: Ahmet Altay <al...@google.com>
Committed: Fri Jan 27 17:04:21 2017 -0800

----------------------------------------------------------------------
 sdks/python/pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/f1b8679c/sdks/python/pom.xml
----------------------------------------------------------------------
diff --git a/sdks/python/pom.xml b/sdks/python/pom.xml
index cc90969..615ddc5 100644
--- a/sdks/python/pom.xml
+++ b/sdks/python/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.beam</groupId>
     <artifactId>beam-sdks-parent</artifactId>
-    <version>0.5.0-incubating-SNAPSHOT</version>
+    <version>0.6.0-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 


[35/50] [abbrv] beam git commit: This closes #1807

Posted by da...@apache.org.
This closes #1807


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

Branch: refs/heads/master
Commit: 1bc685980092e0922504858fa6c08adc8c44acaa
Parents: 52d97e2 e5d8810
Author: Dan Halperin <dh...@google.com>
Authored: Fri Jan 27 14:30:28 2017 -0800
Committer: Dan Halperin <dh...@google.com>
Committed: Fri Jan 27 14:30:28 2017 -0800

----------------------------------------------------------------------
 sdks/python/apache_beam/coders/coders.py |  1 -
 sdks/python/run_pylint.sh                | 44 ++++++++-------------------
 2 files changed, 12 insertions(+), 33 deletions(-)
----------------------------------------------------------------------



[33/50] [abbrv] beam git commit: Closes #1857

Posted by da...@apache.org.
Closes #1857


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

Branch: refs/heads/master
Commit: 52d97e2fc2e383a58969447addd45ebe3eed4f5f
Parents: 3d6f20d 901a14c
Author: Robert Bradshaw <ro...@gmail.com>
Authored: Fri Jan 27 12:00:25 2017 -0800
Committer: Robert Bradshaw <ro...@gmail.com>
Committed: Fri Jan 27 12:00:25 2017 -0800

----------------------------------------------------------------------
 .../clients/dataflow/dataflow_v1b3_client.py    | 578 ++++++++----
 .../clients/dataflow/dataflow_v1b3_messages.py  | 931 +++++++++++++------
 2 files changed, 1075 insertions(+), 434 deletions(-)
----------------------------------------------------------------------



[42/50] [abbrv] beam git commit: Updates places in SDK that creates thread pools.

Posted by da...@apache.org.
Updates places in SDK that creates thread pools.

Moves ThreadPool creation to a util function.
Records and resets logging level due to this being reset by  apitools when used with a ThreadPool.


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

Branch: refs/heads/master
Commit: 51afc1ccfe78a0657b5f9bc139d1d4e7938ed672
Parents: f29527f
Author: Chamikara Jayalath <ch...@google.com>
Authored: Sat Jan 28 08:54:33 2017 -0800
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jan 30 12:43:37 2017 -0800

----------------------------------------------------------------------
 sdks/python/apache_beam/internal/util.py      | 33 ++++++++++++++++++++++
 sdks/python/apache_beam/io/filebasedsource.py | 17 +++--------
 sdks/python/apache_beam/io/fileio.py          | 11 ++------
 3 files changed, 40 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/51afc1cc/sdks/python/apache_beam/internal/util.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/internal/util.py b/sdks/python/apache_beam/internal/util.py
index 2d12d49..5b31e88 100644
--- a/sdks/python/apache_beam/internal/util.py
+++ b/sdks/python/apache_beam/internal/util.py
@@ -17,6 +17,11 @@
 
 """Utility functions used throughout the package."""
 
+import logging
+from multiprocessing.pool import ThreadPool
+import threading
+import weakref
+
 
 class ArgumentPlaceholder(object):
   """A place holder object replacing PValues in argument lists.
@@ -92,3 +97,31 @@ def insert_values_in_args(args, kwargs, values):
       (k, v_iter.next()) if isinstance(v, ArgumentPlaceholder) else (k, v)
       for k, v in sorted(kwargs.iteritems()))
   return (new_args, new_kwargs)
+
+
+def run_using_threadpool(fn_to_execute, inputs, pool_size):
+  """Runs the given function on given inputs using a thread pool.
+
+  Args:
+    fn_to_execute: Function to execute
+    inputs: Inputs on which given function will be executed in parallel.
+    pool_size: Size of thread pool.
+  Returns:
+    Results retrieved after executing the given function on given inputs.
+  """
+
+  # ThreadPool crashes in old versions of Python (< 2.7.5) if created
+  # from a child thread. (http://bugs.python.org/issue10015)
+  if not hasattr(threading.current_thread(), '_children'):
+    threading.current_thread()._children = weakref.WeakKeyDictionary()
+  pool = ThreadPool(min(pool_size, len(inputs)))
+  try:
+    # We record and reset logging level here since 'apitools' library Beam
+    # depends on updates the logging level when used with a threadpool -
+    # https://github.com/google/apitools/issues/141
+    # TODO: Remove this once above issue in 'apitools' is fixed.
+    old_level = logging.getLogger().level
+    return pool.map(fn_to_execute, inputs)
+  finally:
+    pool.terminate()
+    logging.getLogger().setLevel(old_level)

http://git-wip-us.apache.org/repos/asf/beam/blob/51afc1cc/sdks/python/apache_beam/io/filebasedsource.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/io/filebasedsource.py b/sdks/python/apache_beam/io/filebasedsource.py
index 1bfde25..582d673 100644
--- a/sdks/python/apache_beam/io/filebasedsource.py
+++ b/sdks/python/apache_beam/io/filebasedsource.py
@@ -26,11 +26,9 @@ For an example implementation of ``FileBasedSource`` see ``avroio.AvroSource``.
 """
 
 import random
-import threading
-import weakref
-from multiprocessing.pool import ThreadPool
 
 from apache_beam.internal import pickler
+from apache_beam.internal import util
 from apache_beam.io import concat_source
 from apache_beam.io import fileio
 from apache_beam.io import iobase
@@ -158,16 +156,9 @@ class FileBasedSource(iobase.BoundedSource):
       return [fileio.ChannelFactory.size_in_bytes(file_names[0])]
     else:
       if pattern is None:
-        # ThreadPool crashes in old versions of Python (< 2.7.5) if created
-        # from a child thread. (http://bugs.python.org/issue10015)
-        if not hasattr(threading.current_thread(), '_children'):
-          threading.current_thread()._children = weakref.WeakKeyDictionary()
-        pool = ThreadPool(
-            min(MAX_NUM_THREADS_FOR_SIZE_ESTIMATION, len(file_names)))
-        try:
-          return pool.map(fileio.ChannelFactory.size_in_bytes, file_names)
-        finally:
-          pool.terminate()
+        return util.run_using_threadpool(
+            fileio.ChannelFactory.size_in_bytes, file_names,
+            MAX_NUM_THREADS_FOR_SIZE_ESTIMATION)
       else:
         file_sizes = fileio.ChannelFactory.size_of_files_in_glob(pattern,
                                                                  file_names)

http://git-wip-us.apache.org/repos/asf/beam/blob/51afc1cc/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 f67dca9..97cf387 100644
--- a/sdks/python/apache_beam/io/fileio.py
+++ b/sdks/python/apache_beam/io/fileio.py
@@ -22,16 +22,14 @@ import bz2
 import cStringIO
 import glob
 import logging
-from multiprocessing.pool import ThreadPool
 import os
 import re
 import shutil
-import threading
 import time
 import zlib
-import weakref
 
 from apache_beam import coders
+from apache_beam.internal import util
 from apache_beam.io import gcsio
 from apache_beam.io import iobase
 from apache_beam.transforms.display import DisplayDataItem
@@ -663,11 +661,8 @@ class FileSink(iobase.Sink):
           logging.debug('Rename successful: %s -> %s', src, dest)
       return exceptions
 
-    # ThreadPool crashes in old versions of Python (< 2.7.5) if created from a
-    # child thread. (http://bugs.python.org/issue10015)
-    if not hasattr(threading.current_thread(), '_children'):
-      threading.current_thread()._children = weakref.WeakKeyDictionary()
-    exception_batches = ThreadPool(num_threads).map(_rename_batch, batches)
+    exception_batches = util.run_using_threadpool(
+        _rename_batch, batches, num_threads)
 
     all_exceptions = []
     for exceptions in exception_batches:


[25/50] [abbrv] beam git commit: Revert "Revert "Remove dataflow_test.py""

Posted by da...@apache.org.
Revert "Revert "Remove dataflow_test.py""

This reverts commit 96fcc7d31c2540f867c3a73903c2aa99183a6b8b.


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

Branch: refs/heads/master
Commit: 2aa7d47e1491e0601b7b4d1476a8f182b2a14dc3
Parents: 4e1028b
Author: Robert Bradshaw <ro...@gmail.com>
Authored: Tue Jan 24 16:33:55 2017 -0800
Committer: Robert Bradshaw <ro...@google.com>
Committed: Wed Jan 25 16:18:09 2017 -0800

----------------------------------------------------------------------
 sdks/python/apache_beam/dataflow_test.py        | 418 -------------------
 .../apache_beam/transforms/ptransform_test.py   |  67 +++
 .../apache_beam/transforms/sideinputs_test.py   | 208 ++++++++-
 3 files changed, 274 insertions(+), 419 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/2aa7d47e/sdks/python/apache_beam/dataflow_test.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/dataflow_test.py b/sdks/python/apache_beam/dataflow_test.py
deleted file mode 100644
index f410230..0000000
--- a/sdks/python/apache_beam/dataflow_test.py
+++ /dev/null
@@ -1,418 +0,0 @@
-#
-# 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.
-#
-
-"""Integration tests for the dataflow package."""
-
-from __future__ import absolute_import
-
-import logging
-import re
-import unittest
-
-import apache_beam as beam
-from apache_beam.pvalue import AsDict
-from apache_beam.pvalue import AsIter as AllOf
-from apache_beam.pvalue import AsList
-from apache_beam.pvalue import AsSingleton
-from apache_beam.pvalue import EmptySideInput
-from apache_beam.pvalue import SideOutputValue
-from apache_beam.test_pipeline import TestPipeline
-from apache_beam.transforms import Create
-from apache_beam.transforms import DoFn
-from apache_beam.transforms import FlatMap
-from apache_beam.transforms import GroupByKey
-from apache_beam.transforms import Map
-from apache_beam.transforms import ParDo
-from apache_beam.transforms import WindowInto
-from apache_beam.transforms.util import assert_that
-from apache_beam.transforms.util import equal_to
-from apache_beam.transforms.window import IntervalWindow
-from apache_beam.transforms.window import WindowFn
-from nose.plugins.attrib import attr
-
-
-class DataflowTest(unittest.TestCase):
-  """Dataflow integration tests."""
-
-  SAMPLE_DATA = ['aa bb cc aa bb aa \n'] * 10
-  SAMPLE_RESULT = [('cc', 10), ('bb', 20), ('aa', 30)]
-
-  @beam.ptransform_fn
-  def Count(pcoll):  # pylint: disable=invalid-name, no-self-argument
-    """A Count transform: v, ... => (v, n), ..."""
-    return (pcoll
-            | 'AddCount' >> Map(lambda x: (x, 1))
-            | 'GroupCounts' >> GroupByKey()
-            | 'AddCounts' >> Map(lambda (x, ones): (x, sum(ones))))
-
-  @attr('ValidatesRunner')
-  def test_word_count(self):
-    pipeline = TestPipeline()
-    lines = pipeline | 'SomeWords' >> Create(DataflowTest.SAMPLE_DATA)
-    result = (
-        (lines | 'GetWords' >> FlatMap(lambda x: re.findall(r'\w+', x)))
-        .apply('CountWords', DataflowTest.Count))
-    assert_that(result, equal_to(DataflowTest.SAMPLE_RESULT))
-    pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_map(self):
-    pipeline = TestPipeline()
-    lines = pipeline | 'input' >> Create(['a', 'b', 'c'])
-    result = (lines
-              | 'upper' >> Map(str.upper)
-              | 'prefix' >> Map(lambda x, prefix: prefix + x, 'foo-'))
-    assert_that(result, equal_to(['foo-A', 'foo-B', 'foo-C']))
-    pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_par_do_with_side_input_as_arg(self):
-    pipeline = TestPipeline()
-    words_list = ['aa', 'bb', 'cc']
-    words = pipeline | 'SomeWords' >> Create(words_list)
-    prefix = pipeline | 'SomeString' >> Create(['xyz'])  # side in
-    suffix = 'zyx'
-    result = words | FlatMap(
-        'DecorateWords',
-        lambda x, pfx, sfx: ['%s-%s-%s' % (pfx, x, sfx)],
-        AsSingleton(prefix), suffix)
-    assert_that(result, equal_to(['xyz-%s-zyx' % x for x in words_list]))
-    pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_par_do_with_side_input_as_keyword_arg(self):
-    pipeline = TestPipeline()
-    words_list = ['aa', 'bb', 'cc']
-    words = pipeline | 'SomeWords' >> Create(words_list)
-    prefix = 'zyx'
-    suffix = pipeline | 'SomeString' >> Create(['xyz'])  # side in
-    result = words | FlatMap(
-        'DecorateWords',
-        lambda x, pfx, sfx: ['%s-%s-%s' % (pfx, x, sfx)],
-        prefix, sfx=AsSingleton(suffix))
-    assert_that(result, equal_to(['zyx-%s-xyz' % x for x in words_list]))
-    pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_par_do_with_do_fn_object(self):
-    class SomeDoFn(DoFn):
-      """A custom DoFn for a FlatMap transform."""
-
-      def process(self, context, prefix, suffix):
-        return ['%s-%s-%s' % (prefix, context.element, suffix)]
-
-    pipeline = TestPipeline()
-    words_list = ['aa', 'bb', 'cc']
-    words = pipeline | 'SomeWords' >> Create(words_list)
-    prefix = 'zyx'
-    suffix = pipeline | 'SomeString' >> Create(['xyz'])  # side in
-    result = words | 'DecorateWordsDoFn' >> ParDo(
-        SomeDoFn(), prefix, suffix=AsSingleton(suffix))
-    assert_that(result, equal_to(['zyx-%s-xyz' % x for x in words_list]))
-    pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_par_do_with_multiple_outputs_and_using_yield(self):
-    class SomeDoFn(DoFn):
-      """A custom DoFn using yield."""
-
-      def process(self, context):
-        yield context.element
-        if context.element % 2 == 0:
-          yield SideOutputValue('even', context.element)
-        else:
-          yield SideOutputValue('odd', context.element)
-
-    pipeline = TestPipeline()
-    nums = pipeline | 'Some Numbers' >> Create([1, 2, 3, 4])
-    results = nums | ParDo(
-        'ClassifyNumbers', SomeDoFn()).with_outputs('odd', 'even', main='main')
-    assert_that(results.main, equal_to([1, 2, 3, 4]))
-    assert_that(results.odd, equal_to([1, 3]), label='assert:odd')
-    assert_that(results.even, equal_to([2, 4]), label='assert:even')
-    pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_par_do_with_multiple_outputs_and_using_return(self):
-    def some_fn(v):
-      if v % 2 == 0:
-        return [v, SideOutputValue('even', v)]
-      else:
-        return [v, SideOutputValue('odd', v)]
-
-    pipeline = TestPipeline()
-    nums = pipeline | 'Some Numbers' >> Create([1, 2, 3, 4])
-    results = nums | FlatMap(
-        'ClassifyNumbers', some_fn).with_outputs('odd', 'even', main='main')
-    assert_that(results.main, equal_to([1, 2, 3, 4]))
-    assert_that(results.odd, equal_to([1, 3]), label='assert:odd')
-    assert_that(results.even, equal_to([2, 4]), label='assert:even')
-    pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_empty_singleton_side_input(self):
-    pipeline = TestPipeline()
-    pcol = pipeline | 'start' >> Create([1, 2])
-    side = pipeline | 'side' >> Create([])  # Empty side input.
-
-    def my_fn(k, s):
-      v = ('empty' if isinstance(s, EmptySideInput) else 'full')
-      return [(k, v)]
-    result = pcol | 'compute' >> FlatMap(my_fn, AsSingleton(side))
-    assert_that(result, equal_to([(1, 'empty'), (2, 'empty')]))
-    pipeline.run()
-
-  # @attr('ValidatesRunner')
-  # TODO(BEAM-1124): Temporarily disable it due to test failed running on
-  # Dataflow service.
-  def test_multi_valued_singleton_side_input(self):
-    pipeline = TestPipeline()
-    pcol = pipeline | 'start' >> Create([1, 2])
-    side = pipeline | 'side' >> Create([3, 4])  # 2 values in side input.
-    pcol | 'compute' >> FlatMap(lambda x, s: [x * s], AsSingleton(side))  # pylint: disable=expression-not-assigned
-    with self.assertRaises(ValueError):
-      pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_default_value_singleton_side_input(self):
-    pipeline = TestPipeline()
-    pcol = pipeline | 'start' >> Create([1, 2])
-    side = pipeline | 'side' >> Create([])  # 0 values in side input.
-    result = pcol | FlatMap(lambda x, s: [x * s], AsSingleton(side, 10))
-    assert_that(result, equal_to([10, 20]))
-    pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_iterable_side_input(self):
-    pipeline = TestPipeline()
-    pcol = pipeline | 'start' >> Create([1, 2])
-    side = pipeline | 'side' >> Create([3, 4])  # 2 values in side input.
-    result = pcol | FlatMap('compute',
-                            lambda x, s: [x * y for y in s], AllOf(side))
-    assert_that(result, equal_to([3, 4, 6, 8]))
-    pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_undeclared_side_outputs(self):
-    pipeline = TestPipeline()
-    nums = pipeline | 'Some Numbers' >> Create([1, 2, 3, 4])
-    results = nums | FlatMap(
-        'ClassifyNumbers',
-        lambda x: [x, SideOutputValue('even' if x % 2 == 0 else 'odd', x)]
-    ).with_outputs()
-    assert_that(results[None], equal_to([1, 2, 3, 4]))
-    assert_that(results.odd, equal_to([1, 3]), label='assert:odd')
-    assert_that(results.even, equal_to([2, 4]), label='assert:even')
-    pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_empty_side_outputs(self):
-    pipeline = TestPipeline()
-    nums = pipeline | 'Some Numbers' >> Create([1, 3, 5])
-    results = nums | FlatMap(
-        'ClassifyNumbers',
-        lambda x: [x, SideOutputValue('even' if x % 2 == 0 else 'odd', x)]
-    ).with_outputs()
-    assert_that(results[None], equal_to([1, 3, 5]))
-    assert_that(results.odd, equal_to([1, 3, 5]), label='assert:odd')
-    assert_that(results.even, equal_to([]), label='assert:even')
-    pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_as_list_and_as_dict_side_inputs(self):
-    a_list = [5, 1, 3, 2, 9]
-    some_pairs = [('crouton', 17), ('supreme', None)]
-    pipeline = TestPipeline()
-    main_input = pipeline | 'main input' >> Create([1])
-    side_list = pipeline | 'side list' >> Create(a_list)
-    side_pairs = pipeline | 'side pairs' >> Create(some_pairs)
-    results = main_input | FlatMap(
-        'concatenate',
-        lambda x, the_list, the_dict: [[x, the_list, the_dict]],
-        AsList(side_list), AsDict(side_pairs))
-
-    def  matcher(expected_elem, expected_list, expected_pairs):
-      def match(actual):
-        [[actual_elem, actual_list, actual_dict]] = actual
-        equal_to([expected_elem])([actual_elem])
-        equal_to(expected_list)(actual_list)
-        equal_to(expected_pairs)(actual_dict.iteritems())
-      return match
-
-    assert_that(results, matcher(1, a_list, some_pairs))
-    pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_as_singleton_without_unique_labels(self):
-    # This should succeed as calling AsSingleton on the same PCollection twice
-    # with the same defaults will return the same PCollectionView.
-    a_list = [2]
-    pipeline = TestPipeline()
-    main_input = pipeline | 'main input' >> Create([1])
-    side_list = pipeline | 'side list' >> Create(a_list)
-    results = main_input | FlatMap(
-        'test',
-        lambda x, s1, s2: [[x, s1, s2]],
-        AsSingleton(side_list), AsSingleton(side_list))
-
-    def  matcher(expected_elem, expected_singleton):
-      def match(actual):
-        [[actual_elem, actual_singleton1, actual_singleton2]] = actual
-        equal_to([expected_elem])([actual_elem])
-        equal_to([expected_singleton])([actual_singleton1])
-        equal_to([expected_singleton])([actual_singleton2])
-      return match
-
-    assert_that(results, matcher(1, 2))
-    pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_as_singleton_with_different_defaults_without_unique_labels(self):
-    # This should fail as AsSingleton with distinct default values should create
-    # distinct PCollectionViews with the same full_label.
-    a_list = [2]
-    pipeline = TestPipeline()
-    main_input = pipeline | 'main input' >> Create([1])
-    side_list = pipeline | 'side list' >> Create(a_list)
-
-    with self.assertRaises(RuntimeError) as e:
-      _ = main_input | FlatMap(
-          'test',
-          lambda x, s1, s2: [[x, s1, s2]],
-          AsSingleton(side_list), AsSingleton(side_list, default_value=3))
-    self.assertTrue(
-        e.exception.message.startswith(
-            'Transform "ViewAsSingleton(side list.None)" does not have a '
-            'stable unique label.'))
-
-  @attr('ValidatesRunner')
-  def test_as_singleton_with_different_defaults_with_unique_labels(self):
-    a_list = []
-    pipeline = TestPipeline()
-    main_input = pipeline | 'main input' >> Create([1])
-    side_list = pipeline | 'side list' >> Create(a_list)
-    results = main_input | FlatMap(
-        'test',
-        lambda x, s1, s2: [[x, s1, s2]],
-        AsSingleton('si1', side_list, default_value=2),
-        AsSingleton('si2', side_list, default_value=3))
-
-    def  matcher(expected_elem, expected_singleton1, expected_singleton2):
-      def match(actual):
-        [[actual_elem, actual_singleton1, actual_singleton2]] = actual
-        equal_to([expected_elem])([actual_elem])
-        equal_to([expected_singleton1])([actual_singleton1])
-        equal_to([expected_singleton2])([actual_singleton2])
-      return match
-
-    assert_that(results, matcher(1, 2, 3))
-    pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_as_list_without_unique_labels(self):
-    # This should succeed as calling AsList on the same PCollection twice will
-    # return the same PCollectionView.
-    a_list = [1, 2, 3]
-    pipeline = TestPipeline()
-    main_input = pipeline | 'main input' >> Create([1])
-    side_list = pipeline | 'side list' >> Create(a_list)
-    results = main_input | FlatMap(
-        'test',
-        lambda x, ls1, ls2: [[x, ls1, ls2]],
-        AsList(side_list), AsList(side_list))
-
-    def  matcher(expected_elem, expected_list):
-      def match(actual):
-        [[actual_elem, actual_list1, actual_list2]] = actual
-        equal_to([expected_elem])([actual_elem])
-        equal_to(expected_list)(actual_list1)
-        equal_to(expected_list)(actual_list2)
-      return match
-
-    assert_that(results, matcher(1, [1, 2, 3]))
-    pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_as_list_with_unique_labels(self):
-    a_list = [1, 2, 3]
-    pipeline = TestPipeline()
-    main_input = pipeline | 'main input' >> Create([1])
-    side_list = pipeline | 'side list' >> Create(a_list)
-    results = main_input | FlatMap(
-        'test',
-        lambda x, ls1, ls2: [[x, ls1, ls2]],
-        AsList(side_list), AsList(side_list, label='label'))
-
-    def  matcher(expected_elem, expected_list):
-      def match(actual):
-        [[actual_elem, actual_list1, actual_list2]] = actual
-        equal_to([expected_elem])([actual_elem])
-        equal_to(expected_list)(actual_list1)
-        equal_to(expected_list)(actual_list2)
-      return match
-
-    assert_that(results, matcher(1, [1, 2, 3]))
-    pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_as_dict_with_unique_labels(self):
-    some_kvs = [('a', 1), ('b', 2)]
-    pipeline = TestPipeline()
-    main_input = pipeline | 'main input' >> Create([1])
-    side_kvs = pipeline | 'side kvs' >> Create(some_kvs)
-    results = main_input | FlatMap(
-        'test',
-        lambda x, dct1, dct2: [[x, dct1, dct2]],
-        AsDict(side_kvs), AsDict(side_kvs, label='label'))
-
-    def  matcher(expected_elem, expected_kvs):
-      def match(actual):
-        [[actual_elem, actual_dict1, actual_dict2]] = actual
-        equal_to([expected_elem])([actual_elem])
-        equal_to(expected_kvs)(actual_dict1.iteritems())
-        equal_to(expected_kvs)(actual_dict2.iteritems())
-      return match
-
-    assert_that(results, matcher(1, some_kvs))
-    pipeline.run()
-
-  @attr('ValidatesRunner')
-  def test_window_transform(self):
-    class TestWindowFn(WindowFn):
-      """Windowing function adding two disjoint windows to each element."""
-
-      def assign(self, assign_context):
-        _ = assign_context
-        return [IntervalWindow(10, 20), IntervalWindow(20, 30)]
-
-      def merge(self, existing_windows):
-        return existing_windows
-
-    pipeline = TestPipeline()
-    numbers = pipeline | 'KVs' >> Create([(1, 10), (2, 20), (3, 30)])
-    result = (numbers
-              | 'W' >> WindowInto(windowfn=TestWindowFn())
-              | 'G' >> GroupByKey())
-    assert_that(
-        result, equal_to([(1, [10]), (1, [10]), (2, [20]),
-                          (2, [20]), (3, [30]), (3, [30])]))
-    pipeline.run()
-
-
-if __name__ == '__main__':
-  logging.getLogger().setLevel(logging.INFO)
-  unittest.main()

http://git-wip-us.apache.org/repos/asf/beam/blob/2aa7d47e/sdks/python/apache_beam/transforms/ptransform_test.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/transforms/ptransform_test.py b/sdks/python/apache_beam/transforms/ptransform_test.py
index 827bc83..68e4482 100644
--- a/sdks/python/apache_beam/transforms/ptransform_test.py
+++ b/sdks/python/apache_beam/transforms/ptransform_test.py
@@ -24,6 +24,7 @@ import re
 import unittest
 
 import hamcrest as hc
+from nose.plugins.attrib import attr
 
 import apache_beam as beam
 from apache_beam.test_pipeline import TestPipeline
@@ -189,6 +190,72 @@ class PTransformTest(unittest.TestCase):
     assert_that(r2.m, equal_to([3, 4, 5]), label='r2')
     pipeline.run()
 
+  @attr('ValidatesRunner')
+  def test_par_do_with_multiple_outputs_and_using_yield(self):
+    class SomeDoFn(beam.DoFn):
+      """A custom DoFn using yield."""
+
+      def process(self, context):
+        yield context.element
+        if context.element % 2 == 0:
+          yield pvalue.SideOutputValue('even', context.element)
+        else:
+          yield pvalue.SideOutputValue('odd', context.element)
+
+    pipeline = TestPipeline()
+    nums = pipeline | 'Some Numbers' >> beam.Create([1, 2, 3, 4])
+    results = nums | beam.ParDo(
+        'ClassifyNumbers', SomeDoFn()).with_outputs('odd', 'even', main='main')
+    assert_that(results.main, equal_to([1, 2, 3, 4]))
+    assert_that(results.odd, equal_to([1, 3]), label='assert:odd')
+    assert_that(results.even, equal_to([2, 4]), label='assert:even')
+    pipeline.run()
+
+  @attr('ValidatesRunner')
+  def test_par_do_with_multiple_outputs_and_using_return(self):
+    def some_fn(v):
+      if v % 2 == 0:
+        return [v, pvalue.SideOutputValue('even', v)]
+      else:
+        return [v, pvalue.SideOutputValue('odd', v)]
+
+    pipeline = TestPipeline()
+    nums = pipeline | 'Some Numbers' >> beam.Create([1, 2, 3, 4])
+    results = nums | beam.FlatMap(
+        'ClassifyNumbers', some_fn).with_outputs('odd', 'even', main='main')
+    assert_that(results.main, equal_to([1, 2, 3, 4]))
+    assert_that(results.odd, equal_to([1, 3]), label='assert:odd')
+    assert_that(results.even, equal_to([2, 4]), label='assert:even')
+    pipeline.run()
+
+  @attr('ValidatesRunner')
+  def test_undeclared_side_outputs(self):
+    pipeline = TestPipeline()
+    nums = pipeline | 'Some Numbers' >> beam.Create([1, 2, 3, 4])
+    results = nums | beam.FlatMap(
+        'ClassifyNumbers',
+        lambda x: [x,
+                   pvalue.SideOutputValue('even' if x % 2 == 0 else 'odd', x)]
+    ).with_outputs()
+    assert_that(results[None], equal_to([1, 2, 3, 4]))
+    assert_that(results.odd, equal_to([1, 3]), label='assert:odd')
+    assert_that(results.even, equal_to([2, 4]), label='assert:even')
+    pipeline.run()
+
+  @attr('ValidatesRunner')
+  def test_empty_side_outputs(self):
+    pipeline = TestPipeline()
+    nums = pipeline | 'Some Numbers' >> beam.Create([1, 3, 5])
+    results = nums | beam.FlatMap(
+        'ClassifyNumbers',
+        lambda x: [x,
+                   pvalue.SideOutputValue('even' if x % 2 == 0 else 'odd', x)]
+    ).with_outputs()
+    assert_that(results[None], equal_to([1, 3, 5]))
+    assert_that(results.odd, equal_to([1, 3, 5]), label='assert:odd')
+    assert_that(results.even, equal_to([]), label='assert:even')
+    pipeline.run()
+
   def test_do_requires_do_fn_returning_iterable(self):
     # This function is incorrect because it returns an object that isn't an
     # iterable.

http://git-wip-us.apache.org/repos/asf/beam/blob/2aa7d47e/sdks/python/apache_beam/transforms/sideinputs_test.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/transforms/sideinputs_test.py b/sdks/python/apache_beam/transforms/sideinputs_test.py
index a327dc8..9384e7b 100644
--- a/sdks/python/apache_beam/transforms/sideinputs_test.py
+++ b/sdks/python/apache_beam/transforms/sideinputs_test.py
@@ -20,7 +20,10 @@
 import logging
 import unittest
 
+from nose.plugins.attrib import attr
+
 import apache_beam as beam
+from apache_beam.test_pipeline import TestPipeline
 from apache_beam.transforms import window
 from apache_beam.transforms.util import assert_that, equal_to
 
@@ -28,7 +31,7 @@ from apache_beam.transforms.util import assert_that, equal_to
 class SideInputsTest(unittest.TestCase):
 
   def create_pipeline(self):
-    return beam.Pipeline('DirectRunner')
+    return TestPipeline('DirectRunner')
 
   def run_windowed_side_inputs(self, elements, main_window_fn,
                                side_window_fn=None,
@@ -125,6 +128,209 @@ class SideInputsTest(unittest.TestCase):
             (11, {'k11': 'v11'}),
         ])
 
+  @attr('ValidatesRunner')
+  def test_empty_singleton_side_input(self):
+    pipeline = self.create_pipeline()
+    pcol = pipeline | 'start' >> beam.Create([1, 2])
+    side = pipeline | 'side' >> beam.Create([])  # Empty side input.
+
+    def my_fn(k, s):
+      # TODO(robertwb): Should this be an error as in Java?
+      v = ('empty' if isinstance(s, beam.pvalue.EmptySideInput) else 'full')
+      return [(k, v)]
+    result = pcol | 'compute' >> beam.FlatMap(
+        my_fn, beam.pvalue.AsSingleton(side))
+    assert_that(result, equal_to([(1, 'empty'), (2, 'empty')]))
+    pipeline.run()
+
+  # @attr('ValidatesRunner')
+  # TODO(BEAM-1124): Temporarily disable it due to test failed running on
+  # Dataflow service.
+  def test_multi_valued_singleton_side_input(self):
+    pipeline = self.create_pipeline()
+    pcol = pipeline | 'start' >> beam.Create([1, 2])
+    side = pipeline | 'side' >> beam.Create([3, 4])  # 2 values in side input.
+    pcol | 'compute' >> beam.FlatMap(  # pylint: disable=expression-not-assigned
+        lambda x, s: [x * s], beam.pvalue.AsSingleton(side))
+    with self.assertRaises(ValueError):
+      pipeline.run()
+
+  @attr('ValidatesRunner')
+  def test_default_value_singleton_side_input(self):
+    pipeline = self.create_pipeline()
+    pcol = pipeline | 'start' >> beam.Create([1, 2])
+    side = pipeline | 'side' >> beam.Create([])  # 0 values in side input.
+    result = pcol | beam.FlatMap(
+        lambda x, s: [x * s], beam.pvalue.AsSingleton(side, 10))
+    assert_that(result, equal_to([10, 20]))
+    pipeline.run()
+
+  @attr('ValidatesRunner')
+  def test_iterable_side_input(self):
+    pipeline = self.create_pipeline()
+    pcol = pipeline | 'start' >> beam.Create([1, 2])
+    side = pipeline | 'side' >> beam.Create([3, 4])  # 2 values in side input.
+    result = pcol | 'compute' >> beam.FlatMap(
+        lambda x, s: [x * y for y in s],
+        beam.pvalue.AsIter(side))
+    assert_that(result, equal_to([3, 4, 6, 8]))
+    pipeline.run()
+
+  @attr('ValidatesRunner')
+  def test_as_list_and_as_dict_side_inputs(self):
+    a_list = [5, 1, 3, 2, 9]
+    some_pairs = [('crouton', 17), ('supreme', None)]
+    pipeline = self.create_pipeline()
+    main_input = pipeline | 'main input' >> beam.Create([1])
+    side_list = pipeline | 'side list' >> beam.Create(a_list)
+    side_pairs = pipeline | 'side pairs' >> beam.Create(some_pairs)
+    results = main_input | 'concatenate' >> beam.FlatMap(
+        lambda x, the_list, the_dict: [[x, the_list, the_dict]],
+        beam.pvalue.AsList(side_list), beam.pvalue.AsDict(side_pairs))
+
+    def  matcher(expected_elem, expected_list, expected_pairs):
+      def match(actual):
+        [[actual_elem, actual_list, actual_dict]] = actual
+        equal_to([expected_elem])([actual_elem])
+        equal_to(expected_list)(actual_list)
+        equal_to(expected_pairs)(actual_dict.iteritems())
+      return match
+
+    assert_that(results, matcher(1, a_list, some_pairs))
+    pipeline.run()
+
+  @attr('ValidatesRunner')
+  def test_as_singleton_without_unique_labels(self):
+    # This should succeed as calling beam.pvalue.AsSingleton on the same
+    # PCollection twice with the same defaults will return the same
+    # PCollectionView.
+    a_list = [2]
+    pipeline = self.create_pipeline()
+    main_input = pipeline | 'main input' >> beam.Create([1])
+    side_list = pipeline | 'side list' >> beam.Create(a_list)
+    results = main_input | beam.FlatMap(
+        lambda x, s1, s2: [[x, s1, s2]],
+        beam.pvalue.AsSingleton(side_list), beam.pvalue.AsSingleton(side_list))
+
+    def  matcher(expected_elem, expected_singleton):
+      def match(actual):
+        [[actual_elem, actual_singleton1, actual_singleton2]] = actual
+        equal_to([expected_elem])([actual_elem])
+        equal_to([expected_singleton])([actual_singleton1])
+        equal_to([expected_singleton])([actual_singleton2])
+      return match
+
+    assert_that(results, matcher(1, 2))
+    pipeline.run()
+
+  @attr('ValidatesRunner')
+  def test_as_singleton_with_different_defaults_without_unique_labels(self):
+    # This should fail as beam.pvalue.AsSingleton with distinct default values
+    # should beam.Create distinct PCollectionViews with the same full_label.
+    a_list = [2]
+    pipeline = self.create_pipeline()
+    main_input = pipeline | 'main input' >> beam.Create([1])
+    side_list = pipeline | 'side list' >> beam.Create(a_list)
+
+    with self.assertRaises(RuntimeError) as e:
+      _ = main_input | beam.FlatMap(
+          lambda x, s1, s2: [[x, s1, s2]],
+          beam.pvalue.AsSingleton(side_list),
+          beam.pvalue.AsSingleton(side_list, default_value=3))
+    self.assertTrue(
+        e.exception.message.startswith(
+            'Transform "ViewAsSingleton(side list.None)" does not have a '
+            'stable unique label.'))
+
+  @attr('ValidatesRunner')
+  def test_as_singleton_with_different_defaults_with_unique_labels(self):
+    a_list = []
+    pipeline = self.create_pipeline()
+    main_input = pipeline | 'main input' >> beam.Create([1])
+    side_list = pipeline | 'side list' >> beam.Create(a_list)
+    results = main_input | beam.FlatMap(
+        lambda x, s1, s2: [[x, s1, s2]],
+        beam.pvalue.AsSingleton('si1', side_list, default_value=2),
+        beam.pvalue.AsSingleton('si2', side_list, default_value=3))
+
+    def  matcher(expected_elem, expected_singleton1, expected_singleton2):
+      def match(actual):
+        [[actual_elem, actual_singleton1, actual_singleton2]] = actual
+        equal_to([expected_elem])([actual_elem])
+        equal_to([expected_singleton1])([actual_singleton1])
+        equal_to([expected_singleton2])([actual_singleton2])
+      return match
+
+    assert_that(results, matcher(1, 2, 3))
+    pipeline.run()
+
+  @attr('ValidatesRunner')
+  def test_as_list_without_unique_labels(self):
+    # This should succeed as calling beam.pvalue.AsList on the same
+    # PCollection twice will return the same PCollectionView.
+    a_list = [1, 2, 3]
+    pipeline = self.create_pipeline()
+    main_input = pipeline | 'main input' >> beam.Create([1])
+    side_list = pipeline | 'side list' >> beam.Create(a_list)
+    results = main_input | beam.FlatMap(
+        lambda x, ls1, ls2: [[x, ls1, ls2]],
+        beam.pvalue.AsList(side_list), beam.pvalue.AsList(side_list))
+
+    def  matcher(expected_elem, expected_list):
+      def match(actual):
+        [[actual_elem, actual_list1, actual_list2]] = actual
+        equal_to([expected_elem])([actual_elem])
+        equal_to(expected_list)(actual_list1)
+        equal_to(expected_list)(actual_list2)
+      return match
+
+    assert_that(results, matcher(1, [1, 2, 3]))
+    pipeline.run()
+
+  @attr('ValidatesRunner')
+  def test_as_list_with_unique_labels(self):
+    a_list = [1, 2, 3]
+    pipeline = self.create_pipeline()
+    main_input = pipeline | 'main input' >> beam.Create([1])
+    side_list = pipeline | 'side list' >> beam.Create(a_list)
+    results = main_input | beam.FlatMap(
+        lambda x, ls1, ls2: [[x, ls1, ls2]],
+        beam.pvalue.AsList(side_list),
+        beam.pvalue.AsList(side_list, label='label'))
+
+    def  matcher(expected_elem, expected_list):
+      def match(actual):
+        [[actual_elem, actual_list1, actual_list2]] = actual
+        equal_to([expected_elem])([actual_elem])
+        equal_to(expected_list)(actual_list1)
+        equal_to(expected_list)(actual_list2)
+      return match
+
+    assert_that(results, matcher(1, [1, 2, 3]))
+    pipeline.run()
+
+  @attr('ValidatesRunner')
+  def test_as_dict_with_unique_labels(self):
+    some_kvs = [('a', 1), ('b', 2)]
+    pipeline = self.create_pipeline()
+    main_input = pipeline | 'main input' >> beam.Create([1])
+    side_kvs = pipeline | 'side kvs' >> beam.Create(some_kvs)
+    results = main_input | beam.FlatMap(
+        lambda x, dct1, dct2: [[x, dct1, dct2]],
+        beam.pvalue.AsDict(side_kvs),
+        beam.pvalue.AsDict(side_kvs, label='label'))
+
+    def  matcher(expected_elem, expected_kvs):
+      def match(actual):
+        [[actual_elem, actual_dict1, actual_dict2]] = actual
+        equal_to([expected_elem])([actual_elem])
+        equal_to(expected_kvs)(actual_dict1.iteritems())
+        equal_to(expected_kvs)(actual_dict2.iteritems())
+      return match
+
+    assert_that(results, matcher(1, some_kvs))
+    pipeline.run()
+
 
 if __name__ == '__main__':
   logging.getLogger().setLevel(logging.DEBUG)


[05/50] [abbrv] beam git commit: Add some typing to prevent speed regression for old_dofn.

Posted by da...@apache.org.
Add some typing to prevent speed regression for old_dofn.


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

Branch: refs/heads/master
Commit: f68c9dc8d17881c43c31922375fec9593265cc5d
Parents: d0474ab
Author: Robert Bradshaw <ro...@gmail.com>
Authored: Sat Jan 21 20:52:02 2017 -0800
Committer: Robert Bradshaw <ro...@gmail.com>
Committed: Sat Jan 21 20:52:02 2017 -0800

----------------------------------------------------------------------
 sdks/python/apache_beam/runners/common.pxd | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/f68c9dc8/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 06fe434..10d1f96 100644
--- a/sdks/python/apache_beam/runners/common.pxd
+++ b/sdks/python/apache_beam/runners/common.pxd
@@ -36,15 +36,17 @@ cdef class DoFnRunner(Receiver):
   cdef object tagged_receivers
   cdef LoggingContext logging_context
   cdef object step_name
-  cdef object is_new_dofn
+  cdef bint is_new_dofn
   cdef object args
-  cdef object kwargs
+  cdef dict kwargs
   cdef object side_inputs
   cdef bint has_windowed_side_inputs
 
   cdef Receiver main_receivers
 
   cpdef process(self, WindowedValue element)
+  cdef old_dofn_process(self, WindowedValue element)
+  cdef new_dofn_process(self, WindowedValue element)
 
   @cython.locals(windowed_value=WindowedValue)
   cpdef _process_outputs(self, WindowedValue element, results)


[39/50] [abbrv] beam git commit: Update the version.py file to match the latest beam version.

Posted by da...@apache.org.
Update the version.py file to match the latest beam version.


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

Branch: refs/heads/master
Commit: 38575a14e2b17c93de2d0e27fe6213daa7101695
Parents: 4aaaf8f
Author: Ahmet Altay <al...@google.com>
Authored: Mon Jan 30 12:21:28 2017 -0800
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jan 30 12:38:38 2017 -0800

----------------------------------------------------------------------
 sdks/python/apache_beam/version.py | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/38575a14/sdks/python/apache_beam/version.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/version.py b/sdks/python/apache_beam/version.py
index 60d9634..12509fb 100644
--- a/sdks/python/apache_beam/version.py
+++ b/sdks/python/apache_beam/version.py
@@ -21,7 +21,7 @@
 import re
 
 
-__version__ = '0.3.0-incubating.dev'  # TODO: PEP 440 and incubating suffix
+__version__ = '0.6.0.dev'
 
 
 # The following utilities are legacy code from the Maven integration;
@@ -40,7 +40,6 @@ def get_version_from_pom():
     search = pattern.search(pom)
     version = search.group(1)
     version = version.replace("-SNAPSHOT", ".dev")
-    # TODO: PEP 440 and incubating suffix
     return version
 
 


[49/50] [abbrv] beam git commit: Move sdks/python/.gitignore to top-level .gitignore

Posted by da...@apache.org.
Move sdks/python/.gitignore to top-level .gitignore


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

Branch: refs/heads/master
Commit: 2d7ce325f69f42a29ba2cf0438b512a80dc64c4b
Parents: e0ff997
Author: Davor Bonaci <da...@google.com>
Authored: Mon Jan 30 14:44:58 2017 -0800
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jan 30 14:46:48 2017 -0800

----------------------------------------------------------------------
 .gitignore             | 10 ++++++++++
 sdks/python/.gitignore | 11 -----------
 2 files changed, 10 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/2d7ce325/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index 39e2f62..bc9f675 100644
--- a/.gitignore
+++ b/.gitignore
@@ -10,6 +10,16 @@ bin/
 sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/
 sdks/java/maven-archetypes/examples-java8/src/main/resources/archetype-resources/src/
 
+# Ignore files generated by the Python build process.
+*.py[cod]
+*.egg-info/
+.eggs/
+.tox/
+build/
+dist/
+distribute-*
+env/
+
 # Ignore IntelliJ files.
 .idea/
 *.iml

http://git-wip-us.apache.org/repos/asf/beam/blob/2d7ce325/sdks/python/.gitignore
----------------------------------------------------------------------
diff --git a/sdks/python/.gitignore b/sdks/python/.gitignore
deleted file mode 100644
index 05ed303..0000000
--- a/sdks/python/.gitignore
+++ /dev/null
@@ -1,11 +0,0 @@
-*~
-*.py[cod]
-*.egg-info/
-.eggs/
-build/
-dist/
-distribute-*
-env/
-
-# Test files
-.tox/


[20/50] [abbrv] beam git commit: Closes #1836

Posted by da...@apache.org.
Closes #1836


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

Branch: refs/heads/master
Commit: 9540cf1762d8595126a1f96301c35524b0a804c2
Parents: 43cb4d7 19789db
Author: Robert Bradshaw <ro...@gmail.com>
Authored: Tue Jan 24 16:32:37 2017 -0800
Committer: Robert Bradshaw <ro...@gmail.com>
Committed: Tue Jan 24 16:32:37 2017 -0800

----------------------------------------------------------------------
 sdks/python/run_postcommit.sh | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------



[04/50] [abbrv] beam git commit: Closes #1805

Posted by da...@apache.org.
Closes #1805


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

Branch: refs/heads/master
Commit: d0474ab5bf01339fd95b0ec6c1db4b226f868d61
Parents: 946135f 9e272ec
Author: Robert Bradshaw <ro...@gmail.com>
Authored: Sat Jan 21 20:37:08 2017 -0800
Committer: Robert Bradshaw <ro...@gmail.com>
Committed: Sat Jan 21 20:37:08 2017 -0800

----------------------------------------------------------------------
 sdks/python/apache_beam/pipeline_test.py        | 100 ++++++++-
 sdks/python/apache_beam/runners/common.pxd      |   4 +
 sdks/python/apache_beam/runners/common.py       | 221 +++++++++++++------
 .../runners/direct/transform_evaluator.py       |  15 +-
 sdks/python/apache_beam/transforms/core.py      | 113 +++++++++-
 sdks/python/apache_beam/typehints/decorators.py |   2 +-
 sdks/python/apache_beam/typehints/typecheck.py  | 145 ++++++++++++
 7 files changed, 531 insertions(+), 69 deletions(-)
----------------------------------------------------------------------



[28/50] [abbrv] beam git commit: Closes #1776

Posted by da...@apache.org.
Closes #1776


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

Branch: refs/heads/master
Commit: e3849af8c8b0982de07f2c24417042be91474039
Parents: c6420df e4eda3c
Author: Robert Bradshaw <ro...@google.com>
Authored: Thu Jan 26 14:51:57 2017 -0800
Committer: Robert Bradshaw <ro...@google.com>
Committed: Thu Jan 26 14:51:57 2017 -0800

----------------------------------------------------------------------
 sdks/python/apache_beam/io/avroio_test.py |  6 ----
 sdks/python/apache_beam/io/fileio.py      | 10 ++++++-
 sdks/python/apache_beam/io/fileio_test.py |  2 --
 sdks/python/apache_beam/io/iobase.py      | 38 +++++++++++++++++---------
 sdks/python/apache_beam/io/textio.py      | 25 +++++++++--------
 sdks/python/apache_beam/io/textio_test.py | 30 --------------------
 6 files changed, 47 insertions(+), 64 deletions(-)
----------------------------------------------------------------------



[11/50] [abbrv] beam git commit: Closes #1812

Posted by da...@apache.org.
Closes #1812


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

Branch: refs/heads/master
Commit: af49908b8fb5bb34428343218461660d41ead399
Parents: deb2aea 6cb2f37
Author: Robert Bradshaw <ro...@google.com>
Authored: Mon Jan 23 14:37:45 2017 -0800
Committer: Robert Bradshaw <ro...@google.com>
Committed: Mon Jan 23 14:37:45 2017 -0800

----------------------------------------------------------------------
 sdks/python/apache_beam/runners/common.py | 34 ++++++++++++++------------
 1 file changed, 19 insertions(+), 15 deletions(-)
----------------------------------------------------------------------



[22/50] [abbrv] beam git commit: Cleanup tests in pipeline_test.

Posted by da...@apache.org.
Cleanup tests in pipeline_test.

Notably, the runner_name parameter has been obsolete since the removal
of DiskCachedRunnerPipelineTest and is an inferior version of what
TestPipeline provides.


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

Branch: refs/heads/master
Commit: 61d8d3f0690142f6dc87b1484d3ebd148a706837
Parents: 9540cf1
Author: Robert Bradshaw <ro...@gmail.com>
Authored: Sat Jan 21 21:07:39 2017 -0800
Committer: Robert Bradshaw <ro...@google.com>
Committed: Wed Jan 25 12:38:03 2017 -0800

----------------------------------------------------------------------
 sdks/python/apache_beam/pipeline_test.py | 57 ++++++++++-----------------
 1 file changed, 21 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/61d8d3f0/sdks/python/apache_beam/pipeline_test.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/pipeline_test.py b/sdks/python/apache_beam/pipeline_test.py
index 93b68d1..833293f 100644
--- a/sdks/python/apache_beam/pipeline_test.py
+++ b/sdks/python/apache_beam/pipeline_test.py
@@ -38,8 +38,8 @@ from apache_beam.transforms import Read
 from apache_beam.transforms import WindowInto
 from apache_beam.transforms.util import assert_that
 from apache_beam.transforms.util import equal_to
-from apache_beam.transforms.window import IntervalWindow
-from apache_beam.transforms.window import WindowFn
+from apache_beam.transforms.window import SlidingWindows
+from apache_beam.transforms.window import TimestampedValue
 from apache_beam.utils.timestamp import MIN_TIMESTAMP
 
 
@@ -70,9 +70,6 @@ class FakeSource(NativeSource):
 
 class PipelineTest(unittest.TestCase):
 
-  def setUp(self):
-    self.runner_name = 'DirectRunner'
-
   @staticmethod
   def custom_callable(pcoll):
     return pcoll | '+1' >> FlatMap(lambda x: [x + 1])
@@ -103,7 +100,7 @@ class PipelineTest(unittest.TestCase):
       self.leave_composite.append(transform_node)
 
   def test_create(self):
-    pipeline = TestPipeline(runner=self.runner_name)
+    pipeline = TestPipeline()
     pcoll = pipeline | 'label1' >> Create([1, 2, 3])
     assert_that(pcoll, equal_to([1, 2, 3]))
 
@@ -114,19 +111,19 @@ class PipelineTest(unittest.TestCase):
     pipeline.run()
 
   def test_create_singleton_pcollection(self):
-    pipeline = TestPipeline(runner=self.runner_name)
+    pipeline = TestPipeline()
     pcoll = pipeline | 'label' >> Create([[1, 2, 3]])
     assert_that(pcoll, equal_to([[1, 2, 3]]))
     pipeline.run()
 
   def test_read(self):
-    pipeline = TestPipeline(runner=self.runner_name)
+    pipeline = TestPipeline()
     pcoll = pipeline | 'read' >> Read(FakeSource([1, 2, 3]))
     assert_that(pcoll, equal_to([1, 2, 3]))
     pipeline.run()
 
   def test_visit_entire_graph(self):
-    pipeline = Pipeline(self.runner_name)
+    pipeline = Pipeline()
     pcoll1 = pipeline | 'pcoll' >> Create([1, 2, 3])
     pcoll2 = pcoll1 | 'do1' >> FlatMap(lambda x: [x + 1])
     pcoll3 = pcoll2 | 'do2' >> FlatMap(lambda x: [x + 1])
@@ -145,14 +142,14 @@ class PipelineTest(unittest.TestCase):
     self.assertEqual(visitor.leave_composite[0].transform, transform)
 
   def test_apply_custom_transform(self):
-    pipeline = TestPipeline(runner=self.runner_name)
+    pipeline = TestPipeline()
     pcoll = pipeline | 'pcoll' >> Create([1, 2, 3])
     result = pcoll | PipelineTest.CustomTransform()
     assert_that(result, equal_to([2, 3, 4]))
     pipeline.run()
 
   def test_reuse_custom_transform_instance(self):
-    pipeline = Pipeline(self.runner_name)
+    pipeline = Pipeline()
     pcoll1 = pipeline | 'pcoll1' >> Create([1, 2, 3])
     pcoll2 = pipeline | 'pcoll2' >> Create([4, 5, 6])
     transform = PipelineTest.CustomTransform()
@@ -167,7 +164,7 @@ class PipelineTest(unittest.TestCase):
         'pvalue | "label" >> transform')
 
   def test_reuse_cloned_custom_transform_instance(self):
-    pipeline = TestPipeline(runner=self.runner_name)
+    pipeline = TestPipeline()
     pcoll1 = pipeline | 'pc1' >> Create([1, 2, 3])
     pcoll2 = pipeline | 'pc2' >> Create([4, 5, 6])
     transform = PipelineTest.CustomTransform()
@@ -240,7 +237,7 @@ class PipelineTest(unittest.TestCase):
     def raise_exception(exn):
       raise exn
     with self.assertRaises(ValueError):
-      with Pipeline(self.runner_name) as p:
+      with Pipeline() as p:
         # pylint: disable=expression-not-assigned
         p | Create([ValueError]) | Map(raise_exception)
 
@@ -251,15 +248,12 @@ class PipelineTest(unittest.TestCase):
 
 class NewDoFnTest(unittest.TestCase):
 
-  def setUp(self):
-    self.runner_name = 'DirectRunner'
-
   def test_element(self):
     class TestDoFn(NewDoFn):
       def process(self, element):
         yield element + 10
 
-    pipeline = TestPipeline(runner=self.runner_name)
+    pipeline = TestPipeline()
     pcoll = pipeline | 'Create' >> Create([1, 2]) | 'Do' >> ParDo(TestDoFn())
     assert_that(pcoll, equal_to([11, 12]))
     pipeline.run()
@@ -269,7 +263,7 @@ class NewDoFnTest(unittest.TestCase):
       def process(self, element, context=NewDoFn.ContextParam):
         yield context.element + 10
 
-    pipeline = TestPipeline(runner=self.runner_name)
+    pipeline = TestPipeline()
     pcoll = pipeline | 'Create' >> Create([1, 2])| 'Do' >> ParDo(TestDoFn())
     assert_that(pcoll, equal_to([11, 12]))
     pipeline.run()
@@ -307,25 +301,16 @@ class NewDoFnTest(unittest.TestCase):
   def test_window_param(self):
     class TestDoFn(NewDoFn):
       def process(self, element, window=NewDoFn.WindowParam):
-        yield (float(window.start), float(window.end))
-
-    class TestWindowFn(WindowFn):
-      """Windowing function adding two disjoint windows to each element."""
-
-      def assign(self, assign_context):
-        _ = assign_context
-        return [IntervalWindow(10, 20), IntervalWindow(20, 30)]
+        yield (element, (float(window.start), float(window.end)))
 
-      def merge(self, existing_windows):
-        return existing_windows
-
-    pipeline = TestPipeline(runner=self.runner_name)
+    pipeline = TestPipeline()
     pcoll = (pipeline
-             | 'KVs' >> Create([(1, 10), (2, 20)])
-             | 'W' >> WindowInto(windowfn=TestWindowFn())
-             | 'Do' >> ParDo(TestDoFn()))
-    assert_that(pcoll, equal_to([(10.0, 20.0), (10.0, 20.0),
-                                 (20.0, 30.0), (20.0, 30.0)]))
+             | Create([1, 7])
+             | Map(lambda x: TimestampedValue(x, x))
+             | WindowInto(windowfn=SlidingWindows(10, 5))
+             | ParDo(TestDoFn()))
+    assert_that(pcoll, equal_to([(1, (-5, 5)), (1, (0, 10)),
+                                 (7, (0, 10)), (7, (5, 15))]))
     pipeline.run()
 
   def test_timestamp_param(self):
@@ -333,7 +318,7 @@ class NewDoFnTest(unittest.TestCase):
       def process(self, element, timestamp=NewDoFn.TimestampParam):
         yield timestamp
 
-    pipeline = TestPipeline(runner=self.runner_name)
+    pipeline = TestPipeline()
     pcoll = pipeline | 'Create' >> Create([1, 2]) | 'Do' >> ParDo(TestDoFn())
     assert_that(pcoll, equal_to([MIN_TIMESTAMP, MIN_TIMESTAMP]))
     pipeline.run()


[09/50] [abbrv] beam git commit: Closes #1813

Posted by da...@apache.org.
Closes #1813


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

Branch: refs/heads/master
Commit: deb2aea4e96e7e8dc90489f8ad1cf17f1fd1f0ec
Parents: 01b3628 d5b90d8
Author: Robert Bradshaw <ro...@google.com>
Authored: Mon Jan 23 14:36:55 2017 -0800
Committer: Robert Bradshaw <ro...@google.com>
Committed: Mon Jan 23 14:36:55 2017 -0800

----------------------------------------------------------------------
 sdks/python/apache_beam/dataflow_test.py        | 418 -------------------
 .../apache_beam/transforms/ptransform_test.py   |  67 +++
 .../apache_beam/transforms/sideinputs_test.py   | 208 ++++++++-
 3 files changed, 274 insertions(+), 419 deletions(-)
----------------------------------------------------------------------



[24/50] [abbrv] beam git commit: Use a temp directory for requirements cache in test_with_requirements_file

Posted by da...@apache.org.
Use a temp directory for requirements cache in
test_with_requirements_file

The test fails if there are leftover files in the default folder for
requirements cache either from earlier tests, or from the previous
workspaces.


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

Branch: refs/heads/master
Commit: 5787e817a7eda4859963d535df21f2fa00edf8af
Parents: 5924220
Author: Ahmet Altay <al...@google.com>
Authored: Wed Jan 25 09:57:18 2017 -0800
Committer: Robert Bradshaw <ro...@google.com>
Committed: Wed Jan 25 16:16:52 2017 -0800

----------------------------------------------------------------------
 .../python/apache_beam/utils/dependency_test.py | 47 +++++++++++---------
 1 file changed, 27 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/5787e817/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 a484d60..75a89e2 100644
--- a/sdks/python/apache_beam/utils/dependency_test.py
+++ b/sdks/python/apache_beam/utils/dependency_test.py
@@ -106,27 +106,34 @@ class SetupTest(unittest.TestCase):
         dependency.stage_job_resources(options))
 
   def test_with_requirements_file(self):
-    staging_dir = tempfile.mkdtemp()
-    source_dir = tempfile.mkdtemp()
+    try:
+      staging_dir = tempfile.mkdtemp()
+      requirements_cache_dir = tempfile.mkdtemp()
+      source_dir = tempfile.mkdtemp()
 
-    options = PipelineOptions()
-    options.view_as(GoogleCloudOptions).staging_location = staging_dir
-    self.update_options(options)
-    options.view_as(SetupOptions).requirements_file = os.path.join(
-        source_dir, dependency.REQUIREMENTS_FILE)
-    self.create_temp_file(
-        os.path.join(source_dir, dependency.REQUIREMENTS_FILE), 'nothing')
-    self.assertEqual(
-        sorted([dependency.REQUIREMENTS_FILE,
-                'abc.txt', 'def.txt']),
-        sorted(dependency.stage_job_resources(
-            options,
-            populate_requirements_cache=self.populate_requirements_cache)))
-    self.assertTrue(
-        os.path.isfile(
-            os.path.join(staging_dir, dependency.REQUIREMENTS_FILE)))
-    self.assertTrue(os.path.isfile(os.path.join(staging_dir, 'abc.txt')))
-    self.assertTrue(os.path.isfile(os.path.join(staging_dir, 'def.txt')))
+      options = PipelineOptions()
+      options.view_as(GoogleCloudOptions).staging_location = staging_dir
+      self.update_options(options)
+      options.view_as(SetupOptions).requirements_cache = requirements_cache_dir
+      options.view_as(SetupOptions).requirements_file = os.path.join(
+          source_dir, dependency.REQUIREMENTS_FILE)
+      self.create_temp_file(
+          os.path.join(source_dir, dependency.REQUIREMENTS_FILE), 'nothing')
+      self.assertEqual(
+          sorted([dependency.REQUIREMENTS_FILE,
+                  'abc.txt', 'def.txt']),
+          sorted(dependency.stage_job_resources(
+              options,
+              populate_requirements_cache=self.populate_requirements_cache)))
+      self.assertTrue(
+          os.path.isfile(
+              os.path.join(staging_dir, dependency.REQUIREMENTS_FILE)))
+      self.assertTrue(os.path.isfile(os.path.join(staging_dir, 'abc.txt')))
+      self.assertTrue(os.path.isfile(os.path.join(staging_dir, 'def.txt')))
+    finally:
+      shutil.rmtree(staging_dir)
+      shutil.rmtree(requirements_cache_dir)
+      shutil.rmtree(source_dir)
 
   def test_requirements_file_not_present(self):
     staging_dir = tempfile.mkdtemp()


[43/50] [abbrv] beam git commit: This closes #1866

Posted by da...@apache.org.
This closes #1866


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

Branch: refs/heads/master
Commit: 475707f0ffd7bc82ca78fa3f3c9e78f661478b99
Parents: f29527f 51afc1c
Author: Davor Bonaci <da...@google.com>
Authored: Mon Jan 30 12:43:48 2017 -0800
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jan 30 12:43:48 2017 -0800

----------------------------------------------------------------------
 sdks/python/apache_beam/internal/util.py      | 33 ++++++++++++++++++++++
 sdks/python/apache_beam/io/filebasedsource.py | 17 +++--------
 sdks/python/apache_beam/io/fileio.py          | 11 ++------
 3 files changed, 40 insertions(+), 21 deletions(-)
----------------------------------------------------------------------



[07/50] [abbrv] beam git commit: Increments major version used by Dataflow runner to 5

Posted by da...@apache.org.
Increments major version used by Dataflow runner to 5


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

Branch: refs/heads/master
Commit: 1811458b0c33fba0dde909fc655452ad8a37c9f9
Parents: 894461e
Author: Chamikara Jayalath <ch...@google.com>
Authored: Mon Jan 23 10:25:28 2017 -0800
Committer: Chamikara Jayalath <ch...@google.com>
Committed: Mon Jan 23 10:25:28 2017 -0800

----------------------------------------------------------------------
 sdks/python/apache_beam/runners/dataflow_runner.py | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/1811458b/sdks/python/apache_beam/runners/dataflow_runner.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/runners/dataflow_runner.py b/sdks/python/apache_beam/runners/dataflow_runner.py
index 31d3386..abcc764 100644
--- a/sdks/python/apache_beam/runners/dataflow_runner.py
+++ b/sdks/python/apache_beam/runners/dataflow_runner.py
@@ -62,7 +62,7 @@ class DataflowRunner(PipelineRunner):
   # Environment version information. It is passed to the service during a
   # a job submission and is used by the service to establish what features
   # are expected by the workers.
-  BATCH_ENVIRONMENT_MAJOR_VERSION = '4'
+  BATCH_ENVIRONMENT_MAJOR_VERSION = '5'
   STREAMING_ENVIRONMENT_MAJOR_VERSION = '0'
 
   def __init__(self, cache=None, blocking=False):


[36/50] [abbrv] beam git commit: Merge remote-tracking branch 'origin/master' into python-sdk.

Posted by da...@apache.org.
Merge remote-tracking branch 'origin/master' into python-sdk.


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

Branch: refs/heads/master
Commit: c2859a55f89c9807a037adfde9f7e8f506c108ce
Parents: 1bc6859 34b4a6d
Author: Ahmet Altay <al...@google.com>
Authored: Fri Jan 27 16:57:44 2017 -0800
Committer: Ahmet Altay <al...@google.com>
Committed: Fri Jan 27 16:57:44 2017 -0800

----------------------------------------------------------------------
 .jenkins/common_job_properties.groovy           |    9 +-
 ...job_beam_PostCommit_Java_MavenInstall.groovy |    2 +-
 .../job_beam_PreCommit_Java_MavenInstall.groovy |    2 +-
 .../job_beam_Release_NightlySnapshot.groovy     |    2 +-
 .jenkins/job_seed.groovy                        |    2 +-
 .travis/README.md                               |    2 +-
 DISCLAIMER                                      |   10 -
 NOTICE                                          |    4 +-
 README.md                                       |   46 +-
 examples/java/README.md                         |   16 +-
 examples/java/pom.xml                           |   21 +-
 .../beam/examples/DebuggingWordCount.java       |    4 +-
 .../org/apache/beam/examples/WordCount.java     |    6 +-
 .../beam/examples/complete/AutoComplete.java    |    2 +-
 .../org/apache/beam/examples/complete/README.md |   14 +-
 .../apache/beam/examples/complete/TfIdf.java    |    2 +-
 .../examples/complete/TopWikipediaSessions.java |    2 +-
 .../examples/complete/TrafficMaxLaneFlow.java   |    2 +-
 .../beam/examples/complete/TrafficRoutes.java   |    2 +-
 .../examples/cookbook/BigQueryTornadoes.java    |    2 +-
 .../cookbook/CombinePerKeyExamples.java         |    2 +-
 .../org/apache/beam/examples/cookbook/README.md |   14 +-
 .../beam/examples/cookbook/TriggerExample.java  |    4 +-
 .../beam/examples/WindowedWordCountIT.java      |   16 +-
 examples/java8/pom.xml                          |    2 +-
 .../beam/examples/complete/game/GameStats.java  |    7 +-
 .../examples/complete/game/LeaderBoard.java     |    5 +-
 .../beam/examples/complete/game/UserScore.java  |    2 +-
 examples/pom.xml                                |   16 +-
 pom.xml                                         |   41 +-
 runners/apex/README.md                          |    4 +-
 runners/apex/pom.xml                            |    3 +-
 .../beam/runners/apex/ApexPipelineOptions.java  |    7 +-
 .../apache/beam/runners/apex/ApexRunner.java    |   43 +-
 .../beam/runners/apex/ApexYarnLauncher.java     |   23 +-
 .../translation/CreateValuesTranslator.java     |   18 +-
 .../FlattenPCollectionTranslator.java           |   28 +-
 .../apex/translation/GroupByKeyTranslator.java  |    2 +-
 .../translation/ParDoBoundMultiTranslator.java  |   27 +-
 .../apex/translation/ParDoBoundTranslator.java  |    4 +-
 .../apex/translation/TranslationContext.java    |   27 +-
 .../apex/translation/WindowBoundTranslator.java |    8 +-
 .../operators/ApexGroupByKeyOperator.java       |    4 +-
 .../operators/ApexParDoOperator.java            |    6 +-
 .../ApexReadUnboundedInputOperator.java         |   17 +-
 .../beam/runners/apex/ApexRunnerTest.java       |   75 ++
 .../beam/runners/apex/ApexYarnLauncherTest.java |    9 +-
 .../runners/apex/examples/WordCountTest.java    |    2 +-
 .../translation/ParDoBoundTranslatorTest.java   |    6 +-
 .../translation/ReadUnboundTranslatorTest.java  |    8 +-
 .../utils/ApexStateInternalsTest.java           |    2 +-
 .../test/resources/beam-runners-apex.properties |   20 +
 runners/core-java/pom.xml                       |    2 +-
 .../beam/runners/core/AssignWindowsDoFn.java    |    3 +-
 .../apache/beam/runners/core/DoFnAdapters.java  |  343 ++++++
 .../apache/beam/runners/core/DoFnRunner.java    |   21 -
 .../apache/beam/runners/core/DoFnRunners.java   |  138 +--
 .../core/GroupAlsoByWindowViaWindowSetDoFn.java |   10 +-
 .../runners/core/GroupAlsoByWindowsDoFn.java    |    5 +-
 .../beam/runners/core/KeyedWorkItemCoder.java   |    4 +-
 .../core/LateDataDroppingDoFnRunner.java        |    1 -
 .../apache/beam/runners/core/NonEmptyPanes.java |    2 +-
 .../org/apache/beam/runners/core/OldDoFn.java   |  472 ++++++++
 .../runners/core/PerKeyCombineFnRunner.java     |   70 --
 .../runners/core/PerKeyCombineFnRunners.java    |  101 --
 .../beam/runners/core/SimpleDoFnRunner.java     |   63 -
 .../beam/runners/core/SimpleOldDoFnRunner.java  |    7 +-
 .../beam/runners/core/SplittableParDo.java      |    7 -
 .../core/UnboundedReadFromBoundedSource.java    |   14 +-
 .../AfterDelayFromFirstElementStateMachine.java |    2 +-
 .../core/triggers/AfterPaneStateMachine.java    |    2 +-
 .../core/DoFnDelegatingAggregatorTest.java      |  144 +++
 .../core/GroupAlsoByWindowsProperties.java      |    2 +-
 .../runners/core/KeyedWorkItemCoderTest.java    |    6 +
 .../core/LateDataDroppingDoFnRunnerTest.java    |    2 +-
 .../apache/beam/runners/core/NoOpOldDoFn.java   |   72 ++
 .../beam/runners/core/OldDoFnContextTest.java   |   72 ++
 .../apache/beam/runners/core/OldDoFnTest.java   |  192 +++
 .../beam/runners/core/ReduceFnRunnerTest.java   |   12 +-
 .../beam/runners/core/ReduceFnTester.java       |    2 +-
 .../runners/core/SimpleOldDoFnRunnerTest.java   |    2 +-
 .../UnboundedReadFromBoundedSourceTest.java     |   12 +-
 runners/direct-java/pom.xml                     |    3 +-
 .../direct/BoundedReadEvaluatorFactory.java     |   10 +-
 ...ecycleManagerRemovingTransformEvaluator.java |   19 +-
 .../beam/runners/direct/EvaluationContext.java  |    2 +-
 .../direct/ExecutorServiceParallelExecutor.java |    4 +-
 .../runners/direct/FlattenEvaluatorFactory.java |    4 +-
 .../GroupAlsoByWindowEvaluatorFactory.java      |    9 +-
 .../direct/GroupByKeyOnlyEvaluatorFactory.java  |   10 +-
 .../beam/runners/direct/ParDoEvaluator.java     |   16 +-
 .../runners/direct/ParDoEvaluatorFactory.java   |   19 +-
 .../direct/ParDoMultiOverrideFactory.java       |   13 +-
 .../direct/StatefulParDoEvaluatorFactory.java   |   27 +-
 .../direct/TestStreamEvaluatorFactory.java      |    5 +-
 .../direct/UnboundedReadEvaluatorFactory.java   |   22 +-
 .../runners/direct/ViewEvaluatorFactory.java    |    8 +-
 .../beam/runners/direct/WatermarkManager.java   |    6 +-
 .../runners/direct/WindowEvaluatorFactory.java  |    3 +-
 .../runners/direct/AggregatorContainerTest.java |   16 +-
 .../direct/BoundedReadEvaluatorFactoryTest.java |    5 -
 .../CopyOnAccessInMemoryStateInternalsTest.java |    4 +-
 .../runners/direct/DirectGraphVisitorTest.java  |   16 +-
 .../beam/runners/direct/DirectRunnerTest.java   |    5 -
 ...leManagerRemovingTransformEvaluatorTest.java |  103 +-
 .../runners/direct/EvaluationContextTest.java   |    6 +-
 .../beam/runners/direct/ParDoEvaluatorTest.java |    3 +-
 .../StatefulParDoEvaluatorFactoryTest.java      |    4 +-
 .../UnboundedReadEvaluatorFactoryTest.java      |   13 +-
 runners/flink/README.md                         |    6 +-
 runners/flink/examples/pom.xml                  |    2 +-
 .../beam/runners/flink/examples/WordCount.java  |    2 +-
 .../flink/examples/streaming/AutoComplete.java  |    2 +-
 .../examples/streaming/KafkaIOExamples.java     |    4 +-
 .../KafkaWindowedWordCountExample.java          |    2 +-
 .../examples/streaming/WindowedWordCount.java   |    2 +-
 runners/flink/pom.xml                           |    2 +-
 runners/flink/runner/pom.xml                    |    4 +-
 .../runners/flink/FlinkPipelineOptions.java     |    6 +-
 .../runners/flink/OldPerKeyCombineFnRunner.java |   62 +
 .../flink/OldPerKeyCombineFnRunners.java        |  155 +++
 .../FlinkBatchTransformTranslators.java         |   40 +-
 .../FlinkBatchTranslationContext.java           |   21 +-
 .../FlinkStreamingTransformTranslators.java     |   46 +-
 .../FlinkStreamingTranslationContext.java       |   20 +-
 .../functions/FlinkDoFnFunction.java            |    4 +-
 .../FlinkMergingNonShuffleReduceFunction.java   |   10 +-
 .../FlinkMergingPartialReduceFunction.java      |    8 +-
 .../functions/FlinkMergingReduceFunction.java   |    8 +-
 .../functions/FlinkMultiOutputDoFnFunction.java |    4 +-
 .../FlinkMultiOutputProcessContext.java         |    2 +-
 .../functions/FlinkNoElementAssignContext.java  |    2 +-
 .../functions/FlinkPartialReduceFunction.java   |   10 +-
 .../functions/FlinkProcessContextBase.java      |    4 +-
 .../functions/FlinkReduceFunction.java          |   10 +-
 .../FlinkSingleOutputProcessContext.java        |    2 +-
 .../wrappers/streaming/DoFnOperator.java        |    6 +-
 .../streaming/SingletonKeyedWorkItemCoder.java  |   10 +-
 .../wrappers/streaming/WindowDoFnOperator.java  |    2 +-
 .../streaming/io/BoundedSourceWrapper.java      |    2 +-
 .../streaming/io/UnboundedFlinkSink.java        |    6 +
 .../streaming/io/UnboundedSourceWrapper.java    |    2 +-
 .../beam/runners/flink/PipelineOptionsTest.java |   13 +
 .../streaming/FlinkStateInternalsTest.java      |    2 +-
 .../streaming/UnboundedSourceWrapperTest.java   |  464 +++----
 runners/google-cloud-dataflow-java/pom.xml      |   15 +-
 .../beam/runners/dataflow/AssignWindows.java    |   89 ++
 .../dataflow/DataflowAggregatorTransforms.java  |   79 ++
 .../dataflow/DataflowMetricUpdateExtractor.java |  109 ++
 .../runners/dataflow/DataflowPipelineJob.java   |    2 -
 .../dataflow/DataflowPipelineTranslator.java    |  510 +++-----
 .../beam/runners/dataflow/DataflowRunner.java   |  109 +-
 .../DataflowUnboundedReadFromBoundedSource.java |  547 +++++++++
 .../beam/runners/dataflow/ReadTranslator.java   |  102 ++
 .../runners/dataflow/TransformTranslator.java   |  120 ++
 .../dataflow/internal/AssignWindows.java        |   89 --
 .../dataflow/internal/CustomSources.java        |    5 -
 .../internal/DataflowAggregatorTransforms.java  |   79 --
 .../internal/DataflowMetricUpdateExtractor.java |  109 --
 .../DataflowUnboundedReadFromBoundedSource.java |  556 ---------
 .../runners/dataflow/internal/IsmFormat.java    |   20 +-
 .../dataflow/internal/ReadTranslator.java       |  107 --
 .../DataflowPipelineWorkerPoolOptions.java      |   16 +-
 .../beam/runners/dataflow/util/DoFnInfo.java    |   66 +-
 .../beam/runners/dataflow/util/GcsStager.java   |   18 +-
 .../beam/runners/dataflow/util/PackageUtil.java |  352 ++++--
 .../beam/runners/dataflow/dataflow.properties   |    6 +-
 .../dataflow/DataflowPipelineJobTest.java       |   38 +-
 .../DataflowPipelineTranslatorTest.java         |    3 +-
 .../runners/dataflow/DataflowRunnerTest.java    |    8 +-
 ...aflowUnboundedReadFromBoundedSourceTest.java |   79 ++
 ...aflowUnboundedReadFromBoundedSourceTest.java |   83 --
 .../DataflowPipelineDebugOptionsTest.java       |    2 +-
 .../options/DataflowPipelineOptionsTest.java    |    4 +-
 .../options/DataflowProfilingOptionsTest.java   |    4 +-
 .../runners/dataflow/util/PackageUtilTest.java  |   69 +-
 runners/pom.xml                                 |   16 +-
 runners/spark/README.md                         |    8 +-
 runners/spark/pom.xml                           |   29 +-
 .../spark/aggregators/NamedAggregators.java     |    4 +-
 .../coders/BeamSparkRunnerRegistrator.java      |   48 +-
 .../spark/coders/StatelessJavaSerializer.java   |   97 ++
 .../runners/spark/coders/WritableCoder.java     |    4 +-
 .../beam/runners/spark/examples/WordCount.java  |    2 +-
 .../beam/runners/spark/io/MicrobatchSource.java |    9 +-
 .../runners/spark/io/SparkUnboundedSource.java  |  127 +-
 .../spark/stateful/StateSpecFunctions.java      |   37 +-
 .../runners/spark/translation/DoFnFunction.java |   15 +-
 .../spark/translation/EvaluationContext.java    |   83 +-
 .../translation/GroupCombineFunctions.java      |    8 +-
 .../spark/translation/MultiDoFnFunction.java    |   14 +-
 .../translation/SparkAbstractCombineFn.java     |   12 +-
 .../spark/translation/SparkGlobalCombineFn.java |   13 +-
 .../translation/SparkGroupAlsoByWindowFn.java   |    2 +-
 .../spark/translation/SparkKeyedCombineFn.java  |   13 +-
 .../spark/translation/SparkPCollectionView.java |   99 ++
 .../spark/translation/SparkRuntimeContext.java  |   63 +-
 .../spark/translation/TransformTranslator.java  |   67 +-
 .../spark/translation/TranslationUtils.java     |   37 +-
 .../streaming/StreamingTransformTranslator.java |  114 +-
 .../runners/spark/util/BroadcastHelper.java     |  127 --
 .../runners/spark/util/SideInputBroadcast.java  |   77 ++
 .../spark/util/SparkSideInputReader.java        |    8 +-
 .../coders/BeamSparkRunnerRegistratorTest.java  |   57 -
 .../streaming/KafkaStreamingTest.java           |   57 +-
 .../ResumeFromCheckpointStreamingTest.java      |   20 +-
 .../streaming/utils/PAssertStreaming.java       |    4 +-
 sdks/java/build-tools/pom.xml                   |    2 +-
 .../src/main/resources/beam/findbugs-filter.xml |   26 -
 sdks/java/core/pom.xml                          |    2 +-
 .../beam/sdk/annotations/Experimental.java      |    5 +-
 .../org/apache/beam/sdk/coders/AtomicCoder.java |    2 +-
 .../org/apache/beam/sdk/coders/AvroCoder.java   |   30 +-
 .../apache/beam/sdk/coders/BigDecimalCoder.java |    6 +-
 .../beam/sdk/coders/BigEndianIntegerCoder.java  |    7 +
 .../beam/sdk/coders/BigEndianLongCoder.java     |    7 +
 .../apache/beam/sdk/coders/ByteArrayCoder.java  |    7 +
 .../org/apache/beam/sdk/coders/ByteCoder.java   |    7 +
 .../apache/beam/sdk/coders/ByteStringCoder.java |    8 +
 .../java/org/apache/beam/sdk/coders/Coder.java  |    7 +
 .../apache/beam/sdk/coders/CollectionCoder.java |   12 +-
 .../org/apache/beam/sdk/coders/CustomCoder.java |   18 +-
 .../apache/beam/sdk/coders/DelegateCoder.java   |   29 +-
 .../org/apache/beam/sdk/coders/DoubleCoder.java |    7 +
 .../apache/beam/sdk/coders/DurationCoder.java   |    8 +
 .../apache/beam/sdk/coders/InstantCoder.java    |    7 +
 .../apache/beam/sdk/coders/IterableCoder.java   |   12 +-
 .../org/apache/beam/sdk/coders/JAXBCoder.java   |   48 +-
 .../org/apache/beam/sdk/coders/KvCoder.java     |   35 +-
 .../beam/sdk/coders/LengthPrefixCoder.java      |  145 +++
 .../org/apache/beam/sdk/coders/ListCoder.java   |    7 +
 .../org/apache/beam/sdk/coders/MapCoder.java    |   62 +-
 .../apache/beam/sdk/coders/NullableCoder.java   |    6 +
 .../beam/sdk/coders/SerializableCoder.java      |   17 +-
 .../org/apache/beam/sdk/coders/SetCoder.java    |   12 +-
 .../apache/beam/sdk/coders/StandardCoder.java   |   39 +-
 .../beam/sdk/coders/StringDelegateCoder.java    |   16 +-
 .../apache/beam/sdk/coders/StringUtf8Coder.java |    7 +
 .../beam/sdk/coders/TableRowJsonCoder.java      |    7 +
 .../beam/sdk/coders/TextualIntegerCoder.java    |    8 +
 .../org/apache/beam/sdk/coders/VarIntCoder.java |   10 +-
 .../apache/beam/sdk/coders/VarLongCoder.java    |    7 +
 .../org/apache/beam/sdk/coders/VoidCoder.java   |    7 +
 .../beam/sdk/coders/protobuf/ProtoCoder.java    |    8 +-
 .../java/org/apache/beam/sdk/io/AvroSource.java |    5 -
 .../sdk/io/BoundedReadFromUnboundedSource.java  |   79 +-
 .../org/apache/beam/sdk/io/BoundedSource.java   |    8 -
 .../apache/beam/sdk/io/CompressedSource.java    |    8 -
 .../org/apache/beam/sdk/io/CountingSource.java  |    5 -
 .../org/apache/beam/sdk/io/FileSystems.java     |   32 +-
 .../java/org/apache/beam/sdk/io/PubsubIO.java   | 1142 +++++++++---------
 .../apache/beam/sdk/io/PubsubUnboundedSink.java |   88 +-
 .../beam/sdk/io/PubsubUnboundedSource.java      |  104 +-
 .../main/java/org/apache/beam/sdk/io/Read.java  |    7 +-
 .../java/org/apache/beam/sdk/io/TextIO.java     |    5 -
 .../java/org/apache/beam/sdk/io/XmlSource.java  |    5 -
 .../org/apache/beam/sdk/options/GcpOptions.java |   36 +-
 .../org/apache/beam/sdk/options/GcsOptions.java |    4 +-
 .../beam/sdk/options/PipelineOptions.java       |    2 +-
 .../sdk/options/PipelineOptionsFactory.java     |   10 +-
 .../apache/beam/sdk/options/ValueProvider.java  |    6 +-
 .../beam/sdk/runners/TransformHierarchy.java    |   33 +-
 .../testing/FlattenWithHeterogeneousCoders.java |   29 +
 .../org/apache/beam/sdk/testing/PAssert.java    |   12 +-
 .../beam/sdk/testing/RunnableOnService.java     |   14 +-
 .../beam/sdk/testing/SourceTestUtils.java       |    5 -
 .../org/apache/beam/sdk/testing/TestStream.java |    8 +
 .../sdk/testing/UsesUnboundedPCollections.java  |   23 +
 .../beam/sdk/testing/ValueInSingleWindow.java   |    6 +-
 .../sdk/transforms/AggregatorRetriever.java     |   13 +-
 .../beam/sdk/transforms/AppliedPTransform.java  |   11 +-
 .../org/apache/beam/sdk/transforms/Combine.java |  197 +--
 .../apache/beam/sdk/transforms/CombineFns.java  |   14 +-
 .../org/apache/beam/sdk/transforms/Count.java   |    4 +-
 .../org/apache/beam/sdk/transforms/Create.java  |    5 -
 .../sdk/transforms/DelegatingAggregator.java    |    2 +-
 .../beam/sdk/transforms/DoFnAdapters.java       |  504 --------
 .../apache/beam/sdk/transforms/DoFnTester.java  |    7 -
 .../apache/beam/sdk/transforms/GroupByKey.java  |    2 +-
 .../org/apache/beam/sdk/transforms/Max.java     |  124 +-
 .../org/apache/beam/sdk/transforms/Mean.java    |   27 +-
 .../org/apache/beam/sdk/transforms/Min.java     |  122 +-
 .../org/apache/beam/sdk/transforms/OldDoFn.java |  758 ------------
 .../apache/beam/sdk/transforms/PTransform.java  |    9 +-
 .../org/apache/beam/sdk/transforms/ParDo.java   |   46 +-
 .../org/apache/beam/sdk/transforms/Regex.java   |  589 ++++++++-
 .../org/apache/beam/sdk/transforms/Sum.java     |   57 +-
 .../apache/beam/sdk/transforms/ToString.java    |  198 +++
 .../org/apache/beam/sdk/transforms/Top.java     |   27 +-
 .../beam/sdk/transforms/join/CoGbkResult.java   |   35 +-
 .../sdk/transforms/reflect/DoFnInvoker.java     |   20 -
 .../sdk/transforms/reflect/DoFnInvokers.java    |  142 +--
 .../sdk/transforms/reflect/DoFnSignature.java   |   15 +-
 .../windowing/AfterDelayFromFirstElement.java   |    2 +-
 .../sdk/transforms/windowing/AfterPane.java     |    2 +-
 .../sdk/transforms/windowing/GlobalWindow.java  |    6 +
 .../transforms/windowing/IntervalWindow.java    |    4 +-
 .../beam/sdk/transforms/windowing/Window.java   |    3 +-
 .../org/apache/beam/sdk/util/CoderUtils.java    |   28 +-
 .../beam/sdk/util/CombineContextFactory.java    |   18 -
 .../org/apache/beam/sdk/util/DefaultBucket.java |  105 ++
 .../util/EmptyOnDeserializationThreadLocal.java |   39 +
 .../apache/beam/sdk/util/GcpProjectUtil.java    |    2 +-
 .../java/org/apache/beam/sdk/util/GcsUtil.java  |   36 +-
 .../org/apache/beam/sdk/util/NameUtils.java     |  162 +++
 .../org/apache/beam/sdk/util/PropertyNames.java |    1 +
 .../org/apache/beam/sdk/util/PubsubClient.java  |   28 +-
 .../apache/beam/sdk/util/PubsubGrpcClient.java  |    6 +-
 .../apache/beam/sdk/util/PubsubJsonClient.java  |    4 +-
 .../apache/beam/sdk/util/PubsubTestClient.java  |    6 +-
 .../org/apache/beam/sdk/util/StringUtils.java   |  100 --
 .../apache/beam/sdk/util/TimerInternals.java    |    4 +-
 .../org/apache/beam/sdk/util/WindowedValue.java |   23 +-
 .../beam/sdk/util/state/StateContexts.java      |    4 +-
 .../org/apache/beam/sdk/values/PValueBase.java  |    4 +-
 .../beam/sdk/values/TimestampedValue.java       |   10 +-
 .../sdk/AggregatorPipelineExtractorTest.java    |   16 +-
 .../apache/beam/sdk/coders/AvroCoderTest.java   |    7 +
 .../beam/sdk/coders/BigDecimalCoderTest.java    |   46 +-
 .../sdk/coders/BigEndianIntegerCoderTest.java   |    9 +
 .../beam/sdk/coders/BigEndianLongCoderTest.java |    9 +
 .../beam/sdk/coders/ByteArrayCoderTest.java     |    6 +
 .../apache/beam/sdk/coders/ByteCoderTest.java   |    9 +
 .../beam/sdk/coders/ByteStringCoderTest.java    |    8 +
 .../beam/sdk/coders/CoderRegistryTest.java      |    6 +
 .../org/apache/beam/sdk/coders/CoderTest.java   |    8 +
 .../beam/sdk/coders/CollectionCoderTest.java    |   16 +
 .../beam/sdk/coders/DefaultCoderTest.java       |    4 +-
 .../beam/sdk/coders/DelegateCoderTest.java      |   35 +-
 .../apache/beam/sdk/coders/DoubleCoderTest.java |    9 +
 .../beam/sdk/coders/DurationCoderTest.java      |   10 +
 .../beam/sdk/coders/InstantCoderTest.java       |    9 +
 .../beam/sdk/coders/IterableCoderTest.java      |   27 +-
 .../apache/beam/sdk/coders/JAXBCoderTest.java   |   26 +-
 .../org/apache/beam/sdk/coders/KvCoderTest.java |   29 +
 .../beam/sdk/coders/LengthPrefixCoderTest.java  |  129 ++
 .../apache/beam/sdk/coders/ListCoderTest.java   |   16 +-
 .../apache/beam/sdk/coders/MapCoderTest.java    |   21 +-
 .../beam/sdk/coders/NullableCoderTest.java      |   12 +
 .../beam/sdk/coders/SerializableCoderTest.java  |    9 +
 .../apache/beam/sdk/coders/SetCoderTest.java    |   16 +
 .../beam/sdk/coders/StandardCoderTest.java      |   40 +
 .../sdk/coders/StringDelegateCoderTest.java     |   11 +
 .../beam/sdk/coders/StringUtf8CoderTest.java    |    9 +
 .../beam/sdk/coders/TableRowJsonCoderTest.java  |    9 +
 .../sdk/coders/TextualIntegerCoderTest.java     |    9 +
 .../apache/beam/sdk/coders/VarIntCoderTest.java |    9 +
 .../beam/sdk/coders/VarLongCoderTest.java       |    9 +
 .../apache/beam/sdk/coders/VoidCoderTest.java   |   40 +
 .../beam/sdk/io/AvroIOGeneratedClassTest.java   |  285 -----
 .../apache/beam/sdk/io/AvroIOTransformTest.java |  324 +++++
 .../beam/sdk/io/CompressedSourceTest.java       |    5 -
 .../apache/beam/sdk/io/FileBasedSourceTest.java |    5 -
 .../org/apache/beam/sdk/io/FileSystemsTest.java |   33 +-
 .../beam/sdk/io/OffsetBasedSourceTest.java      |    5 -
 .../org/apache/beam/sdk/io/PubsubIOTest.java    |   86 +-
 .../beam/sdk/io/PubsubUnboundedSinkTest.java    |   41 +-
 .../beam/sdk/io/PubsubUnboundedSourceTest.java  |   10 +-
 .../java/org/apache/beam/sdk/io/ReadTest.java   |    5 -
 .../java/org/apache/beam/sdk/io/WriteTest.java  |   10 +-
 .../apache/beam/sdk/options/GcpOptionsTest.java |    4 +-
 .../sdk/options/PipelineOptionsFactoryTest.java |    6 +-
 .../beam/sdk/options/PipelineOptionsTest.java   |    3 +-
 .../beam/sdk/options/ValueProviderTest.java     |   36 +-
 .../sdk/options/ValueProviderUtilsTest.java     |    2 +-
 .../sdk/runners/TransformHierarchyTest.java     |   30 +-
 .../apache/beam/sdk/testing/TestStreamTest.java |    5 +
 .../testing/ValueInSingleWindowCoderTest.java   |    7 +
 .../sdk/transforms/ApproximateUniqueTest.java   |  483 ++++----
 .../beam/sdk/transforms/CombineFnsTest.java     |   20 +-
 .../apache/beam/sdk/transforms/CombineTest.java |   99 +-
 .../apache/beam/sdk/transforms/CountTest.java   |    2 +-
 .../apache/beam/sdk/transforms/CreateTest.java  |    8 -
 .../DoFnDelegatingAggregatorTest.java           |  142 ---
 .../apache/beam/sdk/transforms/DoFnTest.java    |   15 +-
 .../beam/sdk/transforms/DoFnTesterTest.java     |    6 +-
 .../apache/beam/sdk/transforms/FlattenTest.java |   27 +
 .../apache/beam/sdk/transforms/KvSwapTest.java  |   13 +-
 .../org/apache/beam/sdk/transforms/MaxTest.java |   20 +-
 .../apache/beam/sdk/transforms/MeanTest.java    |    7 +-
 .../org/apache/beam/sdk/transforms/MinTest.java |   21 +-
 .../apache/beam/sdk/transforms/NoOpOldDoFn.java |   71 --
 .../beam/sdk/transforms/OldDoFnContextTest.java |   69 --
 .../apache/beam/sdk/transforms/OldDoFnTest.java |  188 ---
 .../apache/beam/sdk/transforms/ParDoTest.java   |   74 +-
 .../apache/beam/sdk/transforms/RegexTest.java   |  127 +-
 .../apache/beam/sdk/transforms/SampleTest.java  |  405 ++++---
 .../beam/sdk/transforms/SimpleStatsFnsTest.java |   36 +-
 .../org/apache/beam/sdk/transforms/SumTest.java |   24 +-
 .../beam/sdk/transforms/ToStringTest.java       |  125 ++
 .../org/apache/beam/sdk/transforms/TopTest.java |   13 +-
 .../apache/beam/sdk/transforms/ViewTest.java    |    2 +-
 .../transforms/join/CoGbkResultCoderTest.java   |   10 +-
 .../sdk/transforms/join/UnionCoderTest.java     |   24 +-
 .../transforms/reflect/DoFnInvokersTest.java    |   42 -
 .../transforms/windowing/GlobalWindowTest.java  |   64 +
 .../apache/beam/sdk/util/CombineFnUtilTest.java |    8 +-
 .../apache/beam/sdk/util/DefaultBucketTest.java |  112 ++
 .../org/apache/beam/sdk/util/GcsUtilTest.java   |   56 +
 .../org/apache/beam/sdk/util/NameUtilsTest.java |  177 +++
 .../beam/sdk/util/PubsubGrpcClientTest.java     |    8 +-
 .../beam/sdk/util/PubsubJsonClientTest.java     |    3 +-
 .../beam/sdk/util/PubsubTestClientTest.java     |    4 +-
 .../beam/sdk/util/SerializableUtilsTest.java    |    4 +-
 .../apache/beam/sdk/util/StringUtilsTest.java   |  100 --
 .../beam/sdk/util/TimerInternalsTest.java       |    5 +
 .../beam/sdk/util/ValueWithRecordIdTest.java    |   34 +
 .../apache/beam/sdk/util/WindowedValueTest.java |   23 +
 .../util/state/InMemoryStateInternalsTest.java  |    2 +-
 .../beam/sdk/util/state/StateTagTest.java       |   11 +-
 .../beam/sdk/values/TimestampedValueTest.java   |   19 +-
 sdks/java/extensions/join-library/README.md     |   10 -
 sdks/java/extensions/join-library/pom.xml       |    2 +-
 sdks/java/extensions/pom.xml                    |    2 +-
 sdks/java/extensions/sorter/pom.xml             |    2 +-
 sdks/java/io/elasticsearch/pom.xml              |  175 +++
 .../sdk/io/elasticsearch/ElasticsearchIO.java   |  819 +++++++++++++
 .../beam/sdk/io/elasticsearch/package-info.java |   20 +
 .../elasticsearch/ElasticSearchIOTestUtils.java |  129 ++
 .../io/elasticsearch/ElasticsearchIOTest.java   |  358 ++++++
 sdks/java/io/google-cloud-platform/pom.xml      |    2 +-
 .../beam/sdk/io/gcp/bigquery/BigQueryIO.java    |  308 +++--
 .../sdk/io/gcp/bigquery/BigQueryServices.java   |   16 +-
 .../io/gcp/bigquery/BigQueryServicesImpl.java   |   76 +-
 .../beam/sdk/io/gcp/bigtable/BigtableIO.java    |    8 -
 .../io/gcp/bigtable/BigtableTestOptions.java    |   37 -
 .../sdk/io/gcp/bigquery/BigQueryIOTest.java     |  205 +++-
 .../gcp/bigquery/BigQueryServicesImplTest.java  |  141 +++
 .../sdk/io/gcp/bigquery/BigQueryUtilTest.java   |    3 +-
 .../sdk/io/gcp/bigtable/BigtableIOTest.java     |    5 +-
 .../io/gcp/bigtable/BigtableTestOptions.java    |   37 +
 sdks/java/io/hdfs/pom.xml                       |    2 +-
 .../beam/sdk/io/hdfs/AvroWrapperCoder.java      |    4 +-
 .../apache/beam/sdk/io/hdfs/HDFSFileSource.java |    5 -
 .../apache/beam/sdk/io/hdfs/WritableCoder.java  |    4 +-
 .../beam/sdk/io/hdfs/AvroWrapperCoderTest.java  |    1 -
 sdks/java/io/jdbc/pom.xml                       |    2 +-
 sdks/java/io/jms/pom.xml                        |    2 +-
 sdks/java/io/kafka/pom.xml                      |    2 +-
 .../org/apache/beam/sdk/io/kafka/KafkaIO.java   |    7 +-
 .../beam/sdk/io/kafka/KafkaRecordCoder.java     |    4 +-
 .../apache/beam/sdk/io/kafka/KafkaIOTest.java   |    9 +-
 .../beam/sdk/io/kafka/KafkaRecordCoderTest.java |   34 +
 sdks/java/io/kinesis/pom.xml                    |    2 +-
 .../beam/sdk/io/kinesis/KinesisRecordCoder.java |    4 +-
 .../beam/sdk/io/kinesis/package-info.java       |    2 +-
 sdks/java/io/mongodb/pom.xml                    |    2 +-
 .../beam/sdk/io/mongodb/MongoDbGridFSIO.java    |    5 -
 .../apache/beam/sdk/io/mongodb/MongoDbIO.java   |    5 -
 sdks/java/io/mqtt/pom.xml                       |  152 +++
 .../org/apache/beam/sdk/io/mqtt/MqttIO.java     |  588 +++++++++
 .../apache/beam/sdk/io/mqtt/package-info.java   |   22 +
 .../org/apache/beam/sdk/io/mqtt/MqttIOTest.java |  197 +++
 sdks/java/io/pom.xml                            |    4 +-
 sdks/java/java8tests/pom.xml                    |    2 +-
 .../maven-archetypes/examples-java8/pom.xml     |    2 +-
 .../main/resources/archetype-resources/pom.xml  |   19 +-
 sdks/java/maven-archetypes/examples/pom.xml     |    2 +-
 .../main/resources/archetype-resources/pom.xml  |   19 +-
 sdks/java/maven-archetypes/pom.xml              |    2 +-
 sdks/java/maven-archetypes/starter/pom.xml      |    2 +-
 .../main/resources/archetype-resources/pom.xml  |    4 +-
 .../resources/projects/basic/reference/pom.xml  |    4 +-
 sdks/java/pom.xml                               |    2 +-
 sdks/pom.xml                                    |   15 +-
 465 files changed, 13882 insertions(+), 8046 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/c2859a55/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/beam/blob/c2859a55/sdks/pom.xml
----------------------------------------------------------------------
diff --cc sdks/pom.xml
index e2dff16,06dbb9b..45d8df0
--- a/sdks/pom.xml
+++ b/sdks/pom.xml
@@@ -73,9 -66,15 +67,16 @@@
              </execution>
            </executions>
          </plugin>
 +
        </plugins>
      </pluginManagement>
+ 
+     <plugins>
+       <plugin>
+         <groupId>org.apache.maven.plugins</groupId>
+         <artifactId>maven-checkstyle-plugin</artifactId>
+       </plugin>
+     </plugins>
    </build>
  
  </project>


[27/50] [abbrv] beam git commit: Fix read/write display data

Posted by da...@apache.org.
Fix read/write display data


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

Branch: refs/heads/master
Commit: e4eda3c335b5767bdaf40b56b2dd5d67d7348f20
Parents: c6420df
Author: Pablo <pa...@google.com>
Authored: Fri Jan 13 11:25:36 2017 -0800
Committer: Robert Bradshaw <ro...@google.com>
Committed: Thu Jan 26 14:51:56 2017 -0800

----------------------------------------------------------------------
 sdks/python/apache_beam/io/avroio_test.py |  6 ----
 sdks/python/apache_beam/io/fileio.py      | 10 ++++++-
 sdks/python/apache_beam/io/fileio_test.py |  2 --
 sdks/python/apache_beam/io/iobase.py      | 38 +++++++++++++++++---------
 sdks/python/apache_beam/io/textio.py      | 25 +++++++++--------
 sdks/python/apache_beam/io/textio_test.py | 30 --------------------
 6 files changed, 47 insertions(+), 64 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/e4eda3c3/sdks/python/apache_beam/io/avroio_test.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/io/avroio_test.py b/sdks/python/apache_beam/io/avroio_test.py
index aed468d..d2fb1d1 100644
--- a/sdks/python/apache_beam/io/avroio_test.py
+++ b/sdks/python/apache_beam/io/avroio_test.py
@@ -196,9 +196,6 @@ class TestAvro(unittest.TestCase):
             'file_pattern',
             'some_avro_sink-%(shard_num)05d-of-%(num_shards)05d.end'),
         DisplayDataItemMatcher(
-            'shards',
-            0),
-        DisplayDataItemMatcher(
             'codec',
             'null'),
         DisplayDataItemMatcher(
@@ -219,9 +216,6 @@ class TestAvro(unittest.TestCase):
             'file_pattern',
             'some_avro_sink-%(shard_num)05d-of-%(num_shards)05d'),
         DisplayDataItemMatcher(
-            'shards',
-            0),
-        DisplayDataItemMatcher(
             'codec',
             'deflate'),
         DisplayDataItemMatcher(

http://git-wip-us.apache.org/repos/asf/beam/blob/e4eda3c3/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 52f31c6..f67dca9 100644
--- a/sdks/python/apache_beam/io/fileio.py
+++ b/sdks/python/apache_beam/io/fileio.py
@@ -547,7 +547,8 @@ class FileSink(iobase.Sink):
 
   def display_data(self):
     return {'shards':
-            DisplayDataItem(self.num_shards, label='Number of Shards'),
+            DisplayDataItem(self.num_shards,
+                            label='Number of Shards').drop_if_default(0),
             'compression':
             DisplayDataItem(str(self.compression_type)),
             'file_pattern':
@@ -787,6 +788,13 @@ class TextFileSink(FileSink):
                       '\'textio.WriteToText()\' instead of directly '
                       'instantiating a TextFileSink object.')
 
+  def display_data(self):
+    dd_parent = super(TextFileSink, self).display_data()
+    dd_parent['append_newline'] = DisplayDataItem(
+        self.append_trailing_newlines,
+        label='Append Trailing New Lines')
+    return dd_parent
+
   def write_encoded_record(self, file_handle, encoded_value):
     """Writes a single encoded record."""
     file_handle.write(encoded_value)

http://git-wip-us.apache.org/repos/asf/beam/blob/e4eda3c3/sdks/python/apache_beam/io/fileio_test.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/io/fileio_test.py b/sdks/python/apache_beam/io/fileio_test.py
index ad77dc5..6c33f53 100644
--- a/sdks/python/apache_beam/io/fileio_test.py
+++ b/sdks/python/apache_beam/io/fileio_test.py
@@ -142,8 +142,6 @@ class TestFileSink(unittest.TestCase):
     dd = DisplayData.create_from(sink)
     expected_items = [
         DisplayDataItemMatcher(
-            'shards', 0),
-        DisplayDataItemMatcher(
             'compression', 'auto'),
         DisplayDataItemMatcher(
             'file_pattern',

http://git-wip-us.apache.org/repos/asf/beam/blob/e4eda3c3/sdks/python/apache_beam/io/iobase.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/io/iobase.py b/sdks/python/apache_beam/io/iobase.py
index 12af3b6..1266ed3 100644
--- a/sdks/python/apache_beam/io/iobase.py
+++ b/sdks/python/apache_beam/io/iobase.py
@@ -759,16 +759,15 @@ class WriteImpl(ptransform.PTransform):
       write_result_coll = (keyed_pcoll
                            | core.WindowInto(window.GlobalWindows())
                            | core.GroupByKey()
-                           | 'WriteBundles' >> core.Map(
-                               _write_keyed_bundle, self.sink,
+                           | 'WriteBundles' >> core.ParDo(
+                               _WriteKeyedBundleDoFn(self.sink),
                                AsSingleton(init_result_coll)))
     else:
       min_shards = 1
       write_result_coll = (pcoll
                            | 'WriteBundles' >>
-                           core.ParDo(
-                               _WriteBundleDoFn(), self.sink,
-                               AsSingleton(init_result_coll))
+                           core.ParDo(_WriteBundleDoFn(self.sink),
+                                      AsSingleton(init_result_coll))
                            | 'Pair' >> core.Map(lambda x: (None, x))
                            | core.WindowInto(window.GlobalWindows())
                            | core.GroupByKey()
@@ -788,12 +787,16 @@ class _WriteBundleDoFn(core.DoFn):
   Opens a writer at the first element and closes the writer at finish_bundle().
   """
 
-  def __init__(self):
+  def __init__(self, sink):
     self.writer = None
+    self.sink = sink
 
-  def process(self, context, sink, init_result):
+  def display_data(self):
+    return {'sink_dd': self.sink}
+
+  def process(self, context, init_result):
     if self.writer is None:
-      self.writer = sink.open_writer(init_result, str(uuid.uuid4()))
+      self.writer = self.sink.open_writer(init_result, str(uuid.uuid4()))
     self.writer.write(context.element)
 
   def finish_bundle(self, context, *args, **kwargs):
@@ -801,11 +804,20 @@ class _WriteBundleDoFn(core.DoFn):
       yield window.TimestampedValue(self.writer.close(), window.MAX_TIMESTAMP)
 
 
-def _write_keyed_bundle(bundle, sink, init_result):
-  writer = sink.open_writer(init_result, str(uuid.uuid4()))
-  for element in bundle[1]:  # values
-    writer.write(element)
-  return window.TimestampedValue(writer.close(), window.MAX_TIMESTAMP)
+class _WriteKeyedBundleDoFn(core.DoFn):
+
+  def __init__(self, sink):
+    self.sink = sink
+
+  def display_data(self):
+    return {'sink_dd': self.sink}
+
+  def process(self, context, init_result):
+    bundle = context.element
+    writer = self.sink.open_writer(init_result, str(uuid.uuid4()))
+    for element in bundle[1]:  # values
+      writer.write(element)
+    return [window.TimestampedValue(writer.close(), window.MAX_TIMESTAMP)]
 
 
 def _finalize_write(_, sink, init_result, write_results, min_shards):

http://git-wip-us.apache.org/repos/asf/beam/blob/e4eda3c3/sdks/python/apache_beam/io/textio.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/io/textio.py b/sdks/python/apache_beam/io/textio.py
index 4cdab12..0a593df 100644
--- a/sdks/python/apache_beam/io/textio.py
+++ b/sdks/python/apache_beam/io/textio.py
@@ -83,6 +83,19 @@ class _TextSource(filebasedsource.FileBasedSource):
     self._coder = coder
     self._buffer_size = buffer_size
 
+  def display_data(self):
+    parent_dd = super(_TextSource, self).display_data()
+    parent_dd['strip_newline'] = DisplayDataItem(
+        self._strip_trailing_newlines,
+        label='Strip Trailing New Lines')
+    parent_dd['buffer_size'] = DisplayDataItem(
+        self._buffer_size,
+        label='Buffer Size')
+    parent_dd['coder'] = DisplayDataItem(
+        self._coder.__class__,
+        label='Coder')
+    return parent_dd
+
   def read_records(self, file_name, range_tracker):
     start_offset = range_tracker.start_position()
     read_buffer = _TextSource.ReadBuffer('', 0)
@@ -252,11 +265,6 @@ class ReadFromText(PTransform):
   def expand(self, pvalue):
     return pvalue.pipeline | Read(self._source)
 
-  def display_data(self):
-    return {'source_dd': self._source,
-            'strip_newline': DisplayDataItem(self._strip_trailing_newlines,
-                                             label='Strip Trailing New Lines')}
-
 
 class WriteToText(PTransform):
   """A PTransform for writing to text files."""
@@ -302,16 +310,9 @@ class WriteToText(PTransform):
           compression.
     """
 
-    self._append_trailing_newlines = append_trailing_newlines
     self._sink = _TextSink(file_path_prefix, file_name_suffix,
                            append_trailing_newlines, num_shards,
                            shard_name_template, coder, compression_type)
 
   def expand(self, pcoll):
     return pcoll | Write(self._sink)
-
-  def display_data(self):
-    return {'sink_dd': self._sink,
-            'append_newline': DisplayDataItem(
-                self._append_trailing_newlines,
-                label='Append Trailing New Lines')}

http://git-wip-us.apache.org/repos/asf/beam/blob/e4eda3c3/sdks/python/apache_beam/io/textio_test.py
----------------------------------------------------------------------
diff --git a/sdks/python/apache_beam/io/textio_test.py b/sdks/python/apache_beam/io/textio_test.py
index 4b85584..07c6d9c 100644
--- a/sdks/python/apache_beam/io/textio_test.py
+++ b/sdks/python/apache_beam/io/textio_test.py
@@ -25,8 +25,6 @@ import os
 import tempfile
 import unittest
 
-import hamcrest as hc
-
 import apache_beam as beam
 import apache_beam.io.source_test_utils as source_test_utils
 
@@ -45,9 +43,6 @@ from apache_beam.io.fileio import CompressionTypes
 
 from apache_beam.test_pipeline import TestPipeline
 
-from apache_beam.transforms.display import DisplayData
-from apache_beam.transforms.display_test import DisplayDataItemMatcher
-
 from apache_beam.transforms.util import assert_that
 from apache_beam.transforms.util import equal_to
 
@@ -294,15 +289,6 @@ class TextSourceTest(unittest.TestCase):
         splits[0].source, splits[0].start_position, splits[0].stop_position,
         perform_multi_threaded_test=False)
 
-  def test_read_display_data(self):
-    read = ReadFromText('prefix', validate=False)
-    dd = DisplayData.create_from(read)
-    expected_items = [
-        DisplayDataItemMatcher('compression', 'auto'),
-        DisplayDataItemMatcher('file_pattern', 'prefix'),
-        DisplayDataItemMatcher('strip_newline', True)]
-    hc.assert_that(dd.items, hc.contains_inanyorder(*expected_items))
-
   def test_dataflow_single_file(self):
     file_name, expected_data = write_data(5)
     assert len(expected_data) == 5
@@ -506,22 +492,6 @@ class TextSinkTest(unittest.TestCase):
     with gzip.GzipFile(self.path, 'r') as f:
       self.assertEqual(f.read().splitlines(), [])
 
-  def test_write_display_data(self):
-    write = WriteToText('prefix')
-    dd = DisplayData.create_from(write)
-    expected_items = [
-        DisplayDataItemMatcher(
-            'append_newline', True),
-        DisplayDataItemMatcher(
-            'compression', 'auto'),
-        DisplayDataItemMatcher(
-            'shards', 0),
-        DisplayDataItemMatcher(
-            'file_pattern',
-            '{}{}'.format('prefix',
-                          '-%(shard_num)05d-of-%(num_shards)05d'))]
-    hc.assert_that(dd.items, hc.contains_inanyorder(*expected_items))
-
   def test_write_dataflow(self):
     pipeline = TestPipeline()
     pcoll = pipeline | beam.core.Create(self.lines)