[GitHub] [beam] y1chi commented on a change in pull request #12727: [BEAM-10844] Add experiment option prebuild_sdk_container to prebuild python sdk container with dependencies.

2020-09-23 Thread GitBox


y1chi commented on a change in pull request #12727:
URL: https://github.com/apache/beam/pull/12727#discussion_r493764967



##
File path: sdks/python/apache_beam/transforms/environments.py
##
@@ -252,6 +254,13 @@ def from_runner_api_parameter(payload, capabilities, 
artifacts, context):
   @classmethod
   def from_options(cls, options):
 # type: (PipelineOptions) -> DockerEnvironment
+if options.view_as(SetupOptions).prebuild_sdk_container_engine:
+  prebuilt_container_image = SdkContainerBuilder.build_container_image(
+  options)
+  return cls.from_container_image(
+  container_image=prebuilt_container_image,
+  artifacts=python_sdk_dependencies(
+  options, skip_boot_dependencies=True))

Review comment:
   sounds good, applied the change.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [beam] y1chi commented on a change in pull request #12727: [BEAM-10844] Add experiment option prebuild_sdk_container to prebuild python sdk container with dependencies.

2020-09-23 Thread GitBox


y1chi commented on a change in pull request #12727:
URL: https://github.com/apache/beam/pull/12727#discussion_r493764637



##
File path: sdks/python/apache_beam/options/pipeline_options.py
##
@@ -1003,6 +1003,13 @@ def _add_argparse_args(cls, parser):
 'environment. Choose the docker build engine of local docker '
 'environment or google cloud build by passing the option '
 'local_docker or cloud_build.'))
+parser.add_argument(
+'--prebuild_sdk_container_base_image',
+default=None,
+help=(
+'The base image to use when pre-building the sdk container image '
+'with dependencies, if not specified, the latest public '

Review comment:
   I agree that we should use a nightly build and we can change to that if 
we build those. Some context is that we may need to use this feature in some 
runner v2 rolling upgrade test and building from dockerfile for the test seems 
not applicable.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [beam] y1chi commented on a change in pull request #12727: [BEAM-10844] Add experiment option prebuild_sdk_container to prebuild python sdk container with dependencies.

2020-09-23 Thread GitBox


y1chi commented on a change in pull request #12727:
URL: https://github.com/apache/beam/pull/12727#discussion_r493760627



##
File path: sdks/python/apache_beam/runners/portability/sdk_container_builder.py
##
@@ -0,0 +1,279 @@
+#
+# 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.
+#
+
+"""SdkContainerBuilder builds the portable SDK container with dependencies.
+
+It copies the right boot dependencies, namely: apache beam sdk, python packages
+from requirements.txt, python packages from extra_packages.txt, workflow
+tarball, into the latest public python sdk container image, and run the
+dependencies installation in advance with the boot program in setup only mode
+to build the new image.
+"""
+
+from __future__ import absolute_import
+
+import json
+import logging
+import os
+import shutil
+import subprocess
+import sys
+import tarfile
+import tempfile
+import time
+import uuid
+
+from google.protobuf.duration_pb2 import Duration
+from google.protobuf.json_format import MessageToJson
+
+from apache_beam.internal.gcp.auth import get_service_credentials
+from apache_beam.internal.http_client import get_new_http
+from apache_beam.io.gcp.internal.clients import storage
+from apache_beam.options.pipeline_options import GoogleCloudOptions
+from apache_beam.options.pipeline_options import PipelineOptions  # pylint: 
disable=unused-import
+from apache_beam.options.pipeline_options import SetupOptions
+from apache_beam.portability import common_urns
+from apache_beam.portability.api import beam_runner_api_pb2
+from apache_beam.runners.portability.stager import Stager
+
+ARTIFACTS_CONTAINER_DIR = '/opt/apache/beam/artifacts'
+ARTIFACTS_MANIFEST_FILE = 'artifacts_info.json'
+SDK_CONTAINER_ENTRYPOINT = '/opt/apache/beam/boot'
+DOCKERFILE_TEMPLATE = (
+"""FROM {base_image}
+RUN mkdir -p {workdir}
+COPY ./* {workdir}/
+RUN {entrypoint} --setup_only --artifacts {workdir}/{manifest_file}
+""")
+
+SOURCE_FOLDER = 'source'
+_LOGGER = logging.getLogger(__name__)
+
+
+class SdkContainerBuilder(object):
+  def __init__(self, options):
+self._options = options
+self._docker_registry_push_url = self._options.view_as(
+SetupOptions).docker_registry_push_url
+self._base_image = (
+self._options.view_as(SetupOptions).prebuild_sdk_container_base_image 
or
+'apache/beam_python%s.%s_sdk:latest' %
+(sys.version_info[0], sys.version_info[1]))
+self._temp_src_dir = None
+
+  def build(self):
+container_id = str(uuid.uuid4())
+container_tag = os.path.join(
+self._docker_registry_push_url or '',
+'beam_python_prebuilt_sdk:%s' % container_id)
+with tempfile.TemporaryDirectory() as temp_folder:
+  self._temp_src_dir = temp_folder
+  self.prepare_dependencies()
+  self.invoke_docker_build_and_push(container_id, container_tag)
+
+return container_tag
+
+  def prepare_dependencies(self):
+with tempfile.TemporaryDirectory() as tmp:
+  resources = Stager.create_job_resources(self._options, tmp)
+  # make a copy of the staged artifacts into the temp source folder.
+  for path, name in resources:
+shutil.copyfile(path, os.path.join(self._temp_src_dir, name))
+  with open(os.path.join(self._temp_src_dir, 'Dockerfile'), 'w') as file:
+file.write(
+DOCKERFILE_TEMPLATE.format(
+base_image=self._base_image,
+workdir=ARTIFACTS_CONTAINER_DIR,
+manifest_file=ARTIFACTS_MANIFEST_FILE,
+entrypoint=SDK_CONTAINER_ENTRYPOINT))
+  self.generate_artifacts_manifests_json_file(resources, 
self._temp_src_dir)
+
+  def invoke_docker_build_and_push(self, container_id, container_tag):
+raise NotImplementedError
+
+  @staticmethod
+  def generate_artifacts_manifests_json_file(resources, temp_dir):
+infos = []
+for _, name in resources:
+  info = beam_runner_api_pb2.ArtifactInformation(
+  type_urn=common_urns.StandardArtifacts.Types.FILE.urn,
+  type_payload=beam_runner_api_pb2.ArtifactFilePayload(
+  path=name).SerializeToString(),
+  )
+  infos.append(json.dumps(MessageToJson(info)))
+with open(os.path.join(temp_dir, ARTIFACTS_MANIFEST_FILE), 'w') as file:
+ 

[GitHub] [beam] y1chi commented on a change in pull request #12727: [BEAM-10844] Add experiment option prebuild_sdk_container to prebuild python sdk container with dependencies.

2020-09-22 Thread GitBox


y1chi commented on a change in pull request #12727:
URL: https://github.com/apache/beam/pull/12727#discussion_r492888461



##
File path: sdks/python/apache_beam/runners/portability/sdk_container_builder.py
##
@@ -0,0 +1,274 @@
+#
+# 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.
+#
+
+"""SdkContainerBuilder builds the portable SDK container with dependencies.
+
+It copies the right boot dependencies, namely: apache beam sdk, python packages
+from requirements.txt, python packages from extra_packages.txt, workflow
+tarball, into the latest public python sdk container image, and run the
+dependencies installation in advance with the boot program in setup only mode
+to build the new image.
+"""
+
+from __future__ import absolute_import
+
+import json
+import logging
+import os
+import shutil
+import subprocess
+import sys
+import tarfile
+import tempfile
+import time
+import uuid
+
+from google.protobuf.duration_pb2 import Duration
+from google.protobuf.json_format import MessageToJson
+
+from apache_beam.internal.gcp.auth import get_service_credentials
+from apache_beam.internal.http_client import get_new_http
+from apache_beam.io.gcp.internal.clients import storage
+from apache_beam.options.pipeline_options import GoogleCloudOptions
+from apache_beam.options.pipeline_options import PipelineOptions  # pylint: 
disable=unused-import
+from apache_beam.options.pipeline_options import SetupOptions
+from apache_beam.portability import common_urns
+from apache_beam.portability.api import beam_runner_api_pb2
+from apache_beam.runners.portability.stager import Stager
+
+ARTIFACTS_CONTAINER_DIR = '/opt/apache/beam/artifacts'
+ARTIFACTS_MANIFEST_FILE = 'artifacts_info.json'
+SDK_CONTAINER_ENTRYPOINT = '/opt/apache/beam/boot'
+DOCKERFILE_TEMPLATE = (
+"""FROM apache/beam_python{major}.{minor}_sdk:latest
+RUN mkdir -p {workdir}
+COPY ./* {workdir}/
+RUN {entrypoint} --setup_only --artifacts {workdir}/{manifest_file}
+""")
+
+SOURCE_FOLDER = 'source'
+_LOGGER = logging.getLogger(__name__)
+
+
+class SdkContainerBuilder(object):
+  def __init__(self, options):
+self._options = options
+self._temp_src_dir = tempfile.mkdtemp()

Review comment:
   done.

##
File path: sdks/python/apache_beam/runners/portability/sdk_container_builder.py
##
@@ -0,0 +1,274 @@
+#
+# 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.
+#
+
+"""SdkContainerBuilder builds the portable SDK container with dependencies.
+
+It copies the right boot dependencies, namely: apache beam sdk, python packages
+from requirements.txt, python packages from extra_packages.txt, workflow
+tarball, into the latest public python sdk container image, and run the
+dependencies installation in advance with the boot program in setup only mode
+to build the new image.
+"""
+
+from __future__ import absolute_import
+
+import json
+import logging
+import os
+import shutil
+import subprocess
+import sys
+import tarfile
+import tempfile
+import time
+import uuid
+
+from google.protobuf.duration_pb2 import Duration
+from google.protobuf.json_format import MessageToJson
+
+from apache_beam.internal.gcp.auth import get_service_credentials
+from apache_beam.internal.http_client import get_new_http
+from apache_beam.io.gcp.internal.clients import storage
+from apache_beam.options.pipeline_options import GoogleCloudOptions
+from apache_beam.options.pipeline_options import PipelineOptions  # pylint: 
disable=unused-import
+from apache_beam.options.pipeline_options import 

[GitHub] [beam] y1chi commented on a change in pull request #12727: [BEAM-10844] Add experiment option prebuild_sdk_container to prebuild python sdk container with dependencies.

2020-09-22 Thread GitBox


y1chi commented on a change in pull request #12727:
URL: https://github.com/apache/beam/pull/12727#discussion_r493106053



##
File path: sdks/python/apache_beam/options/pipeline_options.py
##
@@ -1003,6 +1003,13 @@ def _add_argparse_args(cls, parser):
 'environment. Choose the docker build engine of local docker '
 'environment or google cloud build by passing the option '
 'local_docker or cloud_build.'))
+parser.add_argument(
+'--prebuild_sdk_container_base_image',
+default=None,
+help=(
+'The base image to use when pre-building the sdk container image '
+'with dependencies, if not specified, the latest public '

Review comment:
   For a dev sdk version, can we use the last released version? 





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [beam] y1chi commented on a change in pull request #12727: [BEAM-10844] Add experiment option prebuild_sdk_container to prebuild python sdk container with dependencies.

2020-09-22 Thread GitBox


y1chi commented on a change in pull request #12727:
URL: https://github.com/apache/beam/pull/12727#discussion_r493105561



##
File path: sdks/python/apache_beam/runners/portability/sdk_container_builder.py
##
@@ -0,0 +1,279 @@
+#
+# 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.
+#
+
+"""SdkContainerBuilder builds the portable SDK container with dependencies.
+
+It copies the right boot dependencies, namely: apache beam sdk, python packages
+from requirements.txt, python packages from extra_packages.txt, workflow
+tarball, into the latest public python sdk container image, and run the
+dependencies installation in advance with the boot program in setup only mode
+to build the new image.
+"""
+
+from __future__ import absolute_import
+
+import json
+import logging
+import os
+import shutil
+import subprocess
+import sys
+import tarfile
+import tempfile
+import time
+import uuid
+
+from google.protobuf.duration_pb2 import Duration
+from google.protobuf.json_format import MessageToJson
+
+from apache_beam.internal.gcp.auth import get_service_credentials
+from apache_beam.internal.http_client import get_new_http
+from apache_beam.io.gcp.internal.clients import storage
+from apache_beam.options.pipeline_options import GoogleCloudOptions
+from apache_beam.options.pipeline_options import PipelineOptions  # pylint: 
disable=unused-import
+from apache_beam.options.pipeline_options import SetupOptions
+from apache_beam.portability import common_urns
+from apache_beam.portability.api import beam_runner_api_pb2
+from apache_beam.runners.portability.stager import Stager
+
+ARTIFACTS_CONTAINER_DIR = '/opt/apache/beam/artifacts'
+ARTIFACTS_MANIFEST_FILE = 'artifacts_info.json'
+SDK_CONTAINER_ENTRYPOINT = '/opt/apache/beam/boot'
+DOCKERFILE_TEMPLATE = (
+"""FROM {base_image}
+RUN mkdir -p {workdir}
+COPY ./* {workdir}/
+RUN {entrypoint} --setup_only --artifacts {workdir}/{manifest_file}
+""")
+
+SOURCE_FOLDER = 'source'
+_LOGGER = logging.getLogger(__name__)
+
+
+class SdkContainerBuilder(object):
+  def __init__(self, options):
+self._options = options
+self._docker_registry_push_url = self._options.view_as(
+SetupOptions).docker_registry_push_url
+self._base_image = (
+self._options.view_as(SetupOptions).prebuild_sdk_container_base_image 
or
+'apache/beam_python%s.%s_sdk:latest' %
+(sys.version_info[0], sys.version_info[1]))
+self._temp_src_dir = None
+
+  def build(self):
+container_id = str(uuid.uuid4())
+container_tag = os.path.join(
+self._docker_registry_push_url or '',
+'beam_python_prebuilt_sdk:%s' % container_id)
+with tempfile.TemporaryDirectory() as temp_folder:
+  self._temp_src_dir = temp_folder
+  self.prepare_dependencies()
+  self.invoke_docker_build_and_push(container_id, container_tag)
+
+return container_tag
+
+  def prepare_dependencies(self):
+with tempfile.TemporaryDirectory() as tmp:
+  resources = Stager.create_job_resources(self._options, tmp)
+  # make a copy of the staged artifacts into the temp source folder.
+  for path, name in resources:
+shutil.copyfile(path, os.path.join(self._temp_src_dir, name))
+  with open(os.path.join(self._temp_src_dir, 'Dockerfile'), 'w') as file:
+file.write(
+DOCKERFILE_TEMPLATE.format(
+base_image=self._base_image,
+workdir=ARTIFACTS_CONTAINER_DIR,
+manifest_file=ARTIFACTS_MANIFEST_FILE,
+entrypoint=SDK_CONTAINER_ENTRYPOINT))
+  self.generate_artifacts_manifests_json_file(resources, 
self._temp_src_dir)
+
+  def invoke_docker_build_and_push(self, container_id, container_tag):
+raise NotImplementedError
+
+  @staticmethod
+  def generate_artifacts_manifests_json_file(resources, temp_dir):
+infos = []
+for _, name in resources:
+  info = beam_runner_api_pb2.ArtifactInformation(
+  type_urn=common_urns.StandardArtifacts.Types.FILE.urn,
+  type_payload=beam_runner_api_pb2.ArtifactFilePayload(
+  path=name).SerializeToString(),
+  )
+  infos.append(json.dumps(MessageToJson(info)))
+with open(os.path.join(temp_dir, ARTIFACTS_MANIFEST_FILE), 'w') as file:
+ 

[GitHub] [beam] y1chi commented on a change in pull request #12727: [BEAM-10844] Add experiment option prebuild_sdk_container to prebuild python sdk container with dependencies.

2020-09-22 Thread GitBox


y1chi commented on a change in pull request #12727:
URL: https://github.com/apache/beam/pull/12727#discussion_r493104239



##
File path: sdks/python/apache_beam/runners/portability/sdk_container_builder.py
##
@@ -0,0 +1,279 @@
+#
+# 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.
+#
+
+"""SdkContainerBuilder builds the portable SDK container with dependencies.
+
+It copies the right boot dependencies, namely: apache beam sdk, python packages
+from requirements.txt, python packages from extra_packages.txt, workflow
+tarball, into the latest public python sdk container image, and run the
+dependencies installation in advance with the boot program in setup only mode
+to build the new image.
+"""
+
+from __future__ import absolute_import
+
+import json
+import logging
+import os
+import shutil
+import subprocess
+import sys
+import tarfile
+import tempfile
+import time
+import uuid
+
+from google.protobuf.duration_pb2 import Duration
+from google.protobuf.json_format import MessageToJson
+
+from apache_beam.internal.gcp.auth import get_service_credentials
+from apache_beam.internal.http_client import get_new_http
+from apache_beam.io.gcp.internal.clients import storage
+from apache_beam.options.pipeline_options import GoogleCloudOptions
+from apache_beam.options.pipeline_options import PipelineOptions  # pylint: 
disable=unused-import
+from apache_beam.options.pipeline_options import SetupOptions
+from apache_beam.portability import common_urns
+from apache_beam.portability.api import beam_runner_api_pb2
+from apache_beam.runners.portability.stager import Stager
+
+ARTIFACTS_CONTAINER_DIR = '/opt/apache/beam/artifacts'
+ARTIFACTS_MANIFEST_FILE = 'artifacts_info.json'
+SDK_CONTAINER_ENTRYPOINT = '/opt/apache/beam/boot'
+DOCKERFILE_TEMPLATE = (
+"""FROM {base_image}
+RUN mkdir -p {workdir}
+COPY ./* {workdir}/
+RUN {entrypoint} --setup_only --artifacts {workdir}/{manifest_file}
+""")
+
+SOURCE_FOLDER = 'source'
+_LOGGER = logging.getLogger(__name__)
+
+
+class SdkContainerBuilder(object):
+  def __init__(self, options):
+self._options = options
+self._docker_registry_push_url = self._options.view_as(
+SetupOptions).docker_registry_push_url
+self._base_image = (
+self._options.view_as(SetupOptions).prebuild_sdk_container_base_image 
or
+'apache/beam_python%s.%s_sdk:latest' %
+(sys.version_info[0], sys.version_info[1]))
+self._temp_src_dir = None
+
+  def build(self):
+container_id = str(uuid.uuid4())
+container_tag = os.path.join(
+self._docker_registry_push_url or '',
+'beam_python_prebuilt_sdk:%s' % container_id)
+with tempfile.TemporaryDirectory() as temp_folder:
+  self._temp_src_dir = temp_folder
+  self.prepare_dependencies()
+  self.invoke_docker_build_and_push(container_id, container_tag)
+
+return container_tag
+
+  def prepare_dependencies(self):
+with tempfile.TemporaryDirectory() as tmp:
+  resources = Stager.create_job_resources(self._options, tmp)
+  # make a copy of the staged artifacts into the temp source folder.
+  for path, name in resources:
+shutil.copyfile(path, os.path.join(self._temp_src_dir, name))
+  with open(os.path.join(self._temp_src_dir, 'Dockerfile'), 'w') as file:
+file.write(
+DOCKERFILE_TEMPLATE.format(
+base_image=self._base_image,
+workdir=ARTIFACTS_CONTAINER_DIR,
+manifest_file=ARTIFACTS_MANIFEST_FILE,
+entrypoint=SDK_CONTAINER_ENTRYPOINT))
+  self.generate_artifacts_manifests_json_file(resources, 
self._temp_src_dir)
+
+  def invoke_docker_build_and_push(self, container_id, container_tag):
+raise NotImplementedError
+
+  @staticmethod
+  def generate_artifacts_manifests_json_file(resources, temp_dir):
+infos = []
+for _, name in resources:
+  info = beam_runner_api_pb2.ArtifactInformation(
+  type_urn=common_urns.StandardArtifacts.Types.FILE.urn,
+  type_payload=beam_runner_api_pb2.ArtifactFilePayload(
+  path=name).SerializeToString(),
+  )
+  infos.append(json.dumps(MessageToJson(info)))
+with open(os.path.join(temp_dir, ARTIFACTS_MANIFEST_FILE), 'w') as file:
+ 

[GitHub] [beam] y1chi commented on a change in pull request #12727: [BEAM-10844] Add experiment option prebuild_sdk_container to prebuild python sdk container with dependencies.

2020-09-22 Thread GitBox


y1chi commented on a change in pull request #12727:
URL: https://github.com/apache/beam/pull/12727#discussion_r493104158



##
File path: sdks/python/apache_beam/runners/portability/stager.py
##
@@ -136,6 +137,8 @@ def create_job_resources(options,  # type: PipelineOptions
 only for testing.
   populate_requirements_cache: Callable for populating the requirements
 cache. Used only for testing.
+  skip_boot_dependencies: Skip apache beam sdk, requirements, extra
+packages, workflow tarball installs by sdk boot program.

Review comment:
   sounds much better, thanks.

##
File path: sdks/python/apache_beam/transforms/environments.py
##
@@ -252,6 +254,13 @@ def from_runner_api_parameter(payload, capabilities, 
artifacts, context):
   @classmethod
   def from_options(cls, options):
 # type: (PipelineOptions) -> DockerEnvironment
+if options.view_as(SetupOptions).prebuild_sdk_container_engine:
+  prebuilt_container_image = SdkContainerBuilder.build_container_image(
+  options)
+  return cls.from_container_image(
+  container_image=prebuilt_container_image,
+  artifacts=python_sdk_dependencies(
+  options, skip_boot_dependencies=True))

Review comment:
   I think we still need it for python_sdk_dependencies to produce two 
sets(complete set or the reduced set) of artifacts

##
File path: sdks/python/container/boot.go
##
@@ -203,15 +223,9 @@ func setupAcceptableWheelSpecs() error {
 }
 
 // installSetupPackages installs Beam SDK and user dependencies.
-func installSetupPackages(mds []*jobpb.ArtifactMetadata, workDir string) error 
{
+func installSetupPackages(files []string, workDir string) error {

Review comment:
   done.

##
File path: sdks/python/container/boot.go
##
@@ -30,18 +32,21 @@ import (
"time"
 
"github.com/apache/beam/sdks/go/pkg/beam/artifact"
-   jobpb "github.com/apache/beam/sdks/go/pkg/beam/model/jobmanagement_v1"
pipepb "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
"github.com/apache/beam/sdks/go/pkg/beam/provision"
"github.com/apache/beam/sdks/go/pkg/beam/util/execx"
"github.com/apache/beam/sdks/go/pkg/beam/util/grpcx"
+   "github.com/golang/protobuf/jsonpb"
"github.com/golang/protobuf/proto"
"github.com/nightlyone/lockfile"
 )
 
 var (
acceptableWhlSpecs []string
 
+   setupOnly = flag.Bool("setup_only", false, "Execute boot program in 
setup only mode (optional).")

Review comment:
   done.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [beam] y1chi commented on a change in pull request #12727: [BEAM-10844] Add experiment option prebuild_sdk_container to prebuild python sdk container with dependencies.

2020-09-22 Thread GitBox


y1chi commented on a change in pull request #12727:
URL: https://github.com/apache/beam/pull/12727#discussion_r492888461



##
File path: sdks/python/apache_beam/runners/portability/sdk_container_builder.py
##
@@ -0,0 +1,274 @@
+#
+# 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.
+#
+
+"""SdkContainerBuilder builds the portable SDK container with dependencies.
+
+It copies the right boot dependencies, namely: apache beam sdk, python packages
+from requirements.txt, python packages from extra_packages.txt, workflow
+tarball, into the latest public python sdk container image, and run the
+dependencies installation in advance with the boot program in setup only mode
+to build the new image.
+"""
+
+from __future__ import absolute_import
+
+import json
+import logging
+import os
+import shutil
+import subprocess
+import sys
+import tarfile
+import tempfile
+import time
+import uuid
+
+from google.protobuf.duration_pb2 import Duration
+from google.protobuf.json_format import MessageToJson
+
+from apache_beam.internal.gcp.auth import get_service_credentials
+from apache_beam.internal.http_client import get_new_http
+from apache_beam.io.gcp.internal.clients import storage
+from apache_beam.options.pipeline_options import GoogleCloudOptions
+from apache_beam.options.pipeline_options import PipelineOptions  # pylint: 
disable=unused-import
+from apache_beam.options.pipeline_options import SetupOptions
+from apache_beam.portability import common_urns
+from apache_beam.portability.api import beam_runner_api_pb2
+from apache_beam.runners.portability.stager import Stager
+
+ARTIFACTS_CONTAINER_DIR = '/opt/apache/beam/artifacts'
+ARTIFACTS_MANIFEST_FILE = 'artifacts_info.json'
+SDK_CONTAINER_ENTRYPOINT = '/opt/apache/beam/boot'
+DOCKERFILE_TEMPLATE = (
+"""FROM apache/beam_python{major}.{minor}_sdk:latest
+RUN mkdir -p {workdir}
+COPY ./* {workdir}/
+RUN {entrypoint} --setup_only --artifacts {workdir}/{manifest_file}
+""")
+
+SOURCE_FOLDER = 'source'
+_LOGGER = logging.getLogger(__name__)
+
+
+class SdkContainerBuilder(object):
+  def __init__(self, options):
+self._options = options
+self._temp_src_dir = tempfile.mkdtemp()

Review comment:
   done.

##
File path: sdks/python/apache_beam/runners/portability/sdk_container_builder.py
##
@@ -0,0 +1,274 @@
+#
+# 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.
+#
+
+"""SdkContainerBuilder builds the portable SDK container with dependencies.
+
+It copies the right boot dependencies, namely: apache beam sdk, python packages
+from requirements.txt, python packages from extra_packages.txt, workflow
+tarball, into the latest public python sdk container image, and run the
+dependencies installation in advance with the boot program in setup only mode
+to build the new image.
+"""
+
+from __future__ import absolute_import
+
+import json
+import logging
+import os
+import shutil
+import subprocess
+import sys
+import tarfile
+import tempfile
+import time
+import uuid
+
+from google.protobuf.duration_pb2 import Duration
+from google.protobuf.json_format import MessageToJson
+
+from apache_beam.internal.gcp.auth import get_service_credentials
+from apache_beam.internal.http_client import get_new_http
+from apache_beam.io.gcp.internal.clients import storage
+from apache_beam.options.pipeline_options import GoogleCloudOptions
+from apache_beam.options.pipeline_options import PipelineOptions  # pylint: 
disable=unused-import
+from apache_beam.options.pipeline_options import 

[GitHub] [beam] y1chi commented on a change in pull request #12727: [BEAM-10844] Add experiment option prebuild_sdk_container to prebuild python sdk container with dependencies.

2020-09-21 Thread GitBox


y1chi commented on a change in pull request #12727:
URL: https://github.com/apache/beam/pull/12727#discussion_r492220801



##
File path: sdks/python/apache_beam/transforms/environments.py
##
@@ -252,6 +254,14 @@ def from_runner_api_parameter(payload, capabilities, 
artifacts, context):
   @classmethod
   def from_options(cls, options):
 # type: (PipelineOptions) -> DockerEnvironment
+if options.view_as(DebugOptions).lookup_experiment(
+'prebuild_sdk_container'):
+  prebuilt_container_image = SdkContainerBuilder.build_container_imge(

Review comment:
   Changed to invoking from_options in dataflow runner.

##
File path: sdks/python/apache_beam/runners/portability/sdk_container_builder.py
##
@@ -0,0 +1,275 @@
+#
+# 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.
+#
+
+"""SdkContainerBuilder builds the portable SDK container with dependencies.
+
+It copies the right boot dependencies, namely: apache beam sdk, python packages
+from requirements.txt, python packages from extra_packages.txt, workflow
+tarball, into the latest public python sdk container image, and run the
+dependencies installation in advance with the boot program in setup only mode
+to build the new image.
+"""
+
+from __future__ import absolute_import
+
+import json
+import logging
+import os
+import shutil
+import subprocess
+import sys
+import tarfile
+import tempfile
+import time
+import uuid
+
+from google.protobuf.duration_pb2 import Duration
+from google.protobuf.json_format import MessageToJson
+
+from apache_beam.internal.gcp.auth import get_service_credentials
+from apache_beam.internal.http_client import get_new_http
+from apache_beam.io.gcp.internal.clients import storage
+from apache_beam.options.pipeline_options import DebugOptions
+from apache_beam.options.pipeline_options import GoogleCloudOptions
+from apache_beam.options.pipeline_options import PipelineOptions  # pylint: 
disable=unused-import
+from apache_beam.portability import common_urns
+from apache_beam.portability.api import beam_runner_api_pb2
+from apache_beam.runners.portability.stager import Stager
+
+ARTIFACTS_CONTAINER_DIR = '/opt/apache/beam/artifacts'
+ARTIFACTS_MANIFEST_FILE = 'artifacts_info.json'
+SDK_CONTAINER_ENTRYPOINT = '/opt/apache/beam/boot'
+DOCKERFILE_TEMPLATE = (
+"""FROM apache/beam_python{major}.{minor}_sdk:latest
+RUN mkdir -p {workdir}
+COPY ./* {workdir}/
+RUN {entrypoint} --setup_only --artifacts {workdir}/{manifest_file}
+""")
+
+SOURCE_FOLDER = 'source'
+_LOGGER = logging.getLogger(__name__)
+
+
+class SdkContainerBuilder(object):
+  def __init__(self, options):
+self._options = options
+self._temp_src_dir = tempfile.mkdtemp()
+self._docker_registry_push_url = self._options.view_as(
+DebugOptions).lookup_experiment('docker_registry_push_url')
+
+  def build(self):
+container_id = str(uuid.uuid4())
+container_tag = os.path.join(
+self._docker_registry_push_url or '',
+'beam_python_prebuilt_sdk:%s' % container_id)
+self.prepare_dependencies()
+self.invoke_docker_build_and_push(container_id, container_tag)
+
+return container_tag
+
+  def prepare_dependencies(self):
+tmp = tempfile.mkdtemp()
+resources = Stager.create_job_resources(self._options, tmp)
+# make a copy of the staged artifacts into the temp source folder.
+for path, name in resources:
+  shutil.copyfile(path, os.path.join(self._temp_src_dir, name))
+with open(os.path.join(self._temp_src_dir, 'Dockerfile'), 'w') as file:
+  file.write(
+  DOCKERFILE_TEMPLATE.format(
+  major=sys.version_info[0],
+  minor=sys.version_info[1],
+  workdir=ARTIFACTS_CONTAINER_DIR,
+  manifest_file=ARTIFACTS_MANIFEST_FILE,
+  entrypoint=SDK_CONTAINER_ENTRYPOINT))
+self.generate_artifacts_manifests_json_file(resources, self._temp_src_dir)
+
+  def invoke_docker_build_and_push(self, container_id, container_tag):
+raise NotImplementedError
+
+  @staticmethod
+  def generate_artifacts_manifests_json_file(resources, temp_dir):
+infos = []
+for _, name in resources:
+  info = beam_runner_api_pb2.ArtifactInformation(
+  

[GitHub] [beam] y1chi commented on a change in pull request #12727: [BEAM-10844] Add experiment option prebuild_sdk_container to prebuild python sdk container with dependencies.

2020-09-21 Thread GitBox


y1chi commented on a change in pull request #12727:
URL: https://github.com/apache/beam/pull/12727#discussion_r492290645



##
File path: sdks/python/apache_beam/runners/portability/stager.py
##
@@ -119,6 +119,7 @@ def create_job_resources(options,  # type: PipelineOptions
temp_dir,  # type: str
build_setup_args=None,  # type: Optional[List[str]]
populate_requirements_cache=None,  # type: 
Optional[str]
+   skip_boot_dependencies=False, # type: Optional[bool]

Review comment:
   Sounds good to me. The pickled main session is expected to be stored in 
semi persistent directory and I believe that is re-mounted at container launch 
time right now. Should we add another flag to boot.go to specify a separate 
path for prestaged picked main session directory?





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [beam] y1chi commented on a change in pull request #12727: [BEAM-10844] Add experiment option prebuild_sdk_container to prebuild python sdk container with dependencies.

2020-09-21 Thread GitBox


y1chi commented on a change in pull request #12727:
URL: https://github.com/apache/beam/pull/12727#discussion_r492221892



##
File path: sdks/python/apache_beam/runners/dataflow/dataflow_runner.py
##
@@ -473,10 +474,19 @@ def run_pipeline(self, pipeline, options):
 
 use_fnapi = apiclient._use_fnapi(options)
 from apache_beam.transforms import environments
-self._default_environment = (
-environments.DockerEnvironment.from_container_image(
-apiclient.get_container_image_from_options(options),
-artifacts=environments.python_sdk_dependencies(options)))
+if options.view_as(DebugOptions).lookup_experiment(
+'prebuild_sdk_container'):

Review comment:
   done.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [beam] y1chi commented on a change in pull request #12727: [BEAM-10844] Add experiment option prebuild_sdk_container to prebuild python sdk container with dependencies.

2020-09-21 Thread GitBox


y1chi commented on a change in pull request #12727:
URL: https://github.com/apache/beam/pull/12727#discussion_r492221619



##
File path: sdks/python/apache_beam/runners/portability/stager.py
##
@@ -119,6 +119,7 @@ def create_job_resources(options,  # type: PipelineOptions
temp_dir,  # type: str
build_setup_args=None,  # type: Optional[List[str]]
populate_requirements_cache=None,  # type: 
Optional[str]
+   skip_boot_dependencies=False, # type: Optional[bool]

Review comment:
   There are other dependencies that doesn't seem to be handled explicitly 
by the boot sequence, for example pickled main session and java jars for x-lang.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [beam] y1chi commented on a change in pull request #12727: [BEAM-10844] Add experiment option prebuild_sdk_container to prebuild python sdk container with dependencies.

2020-09-21 Thread GitBox


y1chi commented on a change in pull request #12727:
URL: https://github.com/apache/beam/pull/12727#discussion_r492220801



##
File path: sdks/python/apache_beam/transforms/environments.py
##
@@ -252,6 +254,14 @@ def from_runner_api_parameter(payload, capabilities, 
artifacts, context):
   @classmethod
   def from_options(cls, options):
 # type: (PipelineOptions) -> DockerEnvironment
+if options.view_as(DebugOptions).lookup_experiment(
+'prebuild_sdk_container'):
+  prebuilt_container_image = SdkContainerBuilder.build_container_imge(

Review comment:
   Changed to invoking from_options in dataflow runner.

##
File path: sdks/python/apache_beam/runners/portability/sdk_container_builder.py
##
@@ -0,0 +1,275 @@
+#
+# 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.
+#
+
+"""SdkContainerBuilder builds the portable SDK container with dependencies.
+
+It copies the right boot dependencies, namely: apache beam sdk, python packages
+from requirements.txt, python packages from extra_packages.txt, workflow
+tarball, into the latest public python sdk container image, and run the
+dependencies installation in advance with the boot program in setup only mode
+to build the new image.
+"""
+
+from __future__ import absolute_import
+
+import json
+import logging
+import os
+import shutil
+import subprocess
+import sys
+import tarfile
+import tempfile
+import time
+import uuid
+
+from google.protobuf.duration_pb2 import Duration
+from google.protobuf.json_format import MessageToJson
+
+from apache_beam.internal.gcp.auth import get_service_credentials
+from apache_beam.internal.http_client import get_new_http
+from apache_beam.io.gcp.internal.clients import storage
+from apache_beam.options.pipeline_options import DebugOptions
+from apache_beam.options.pipeline_options import GoogleCloudOptions
+from apache_beam.options.pipeline_options import PipelineOptions  # pylint: 
disable=unused-import
+from apache_beam.portability import common_urns
+from apache_beam.portability.api import beam_runner_api_pb2
+from apache_beam.runners.portability.stager import Stager
+
+ARTIFACTS_CONTAINER_DIR = '/opt/apache/beam/artifacts'
+ARTIFACTS_MANIFEST_FILE = 'artifacts_info.json'
+SDK_CONTAINER_ENTRYPOINT = '/opt/apache/beam/boot'
+DOCKERFILE_TEMPLATE = (
+"""FROM apache/beam_python{major}.{minor}_sdk:latest
+RUN mkdir -p {workdir}
+COPY ./* {workdir}/
+RUN {entrypoint} --setup_only --artifacts {workdir}/{manifest_file}
+""")
+
+SOURCE_FOLDER = 'source'
+_LOGGER = logging.getLogger(__name__)
+
+
+class SdkContainerBuilder(object):
+  def __init__(self, options):
+self._options = options
+self._temp_src_dir = tempfile.mkdtemp()
+self._docker_registry_push_url = self._options.view_as(
+DebugOptions).lookup_experiment('docker_registry_push_url')
+
+  def build(self):
+container_id = str(uuid.uuid4())
+container_tag = os.path.join(
+self._docker_registry_push_url or '',
+'beam_python_prebuilt_sdk:%s' % container_id)
+self.prepare_dependencies()
+self.invoke_docker_build_and_push(container_id, container_tag)
+
+return container_tag
+
+  def prepare_dependencies(self):
+tmp = tempfile.mkdtemp()
+resources = Stager.create_job_resources(self._options, tmp)
+# make a copy of the staged artifacts into the temp source folder.
+for path, name in resources:
+  shutil.copyfile(path, os.path.join(self._temp_src_dir, name))
+with open(os.path.join(self._temp_src_dir, 'Dockerfile'), 'w') as file:
+  file.write(
+  DOCKERFILE_TEMPLATE.format(
+  major=sys.version_info[0],
+  minor=sys.version_info[1],
+  workdir=ARTIFACTS_CONTAINER_DIR,
+  manifest_file=ARTIFACTS_MANIFEST_FILE,
+  entrypoint=SDK_CONTAINER_ENTRYPOINT))
+self.generate_artifacts_manifests_json_file(resources, self._temp_src_dir)
+
+  def invoke_docker_build_and_push(self, container_id, container_tag):
+raise NotImplementedError
+
+  @staticmethod
+  def generate_artifacts_manifests_json_file(resources, temp_dir):
+infos = []
+for _, name in resources:
+  info = beam_runner_api_pb2.ArtifactInformation(
+  

[GitHub] [beam] y1chi commented on a change in pull request #12727: [BEAM-10844] Add experiment option prebuild_sdk_container to prebuild python sdk container with dependencies.

2020-09-18 Thread GitBox


y1chi commented on a change in pull request #12727:
URL: https://github.com/apache/beam/pull/12727#discussion_r491178572



##
File path: sdks/python/apache_beam/runners/dataflow/dataflow_runner.py
##
@@ -473,10 +474,19 @@ def run_pipeline(self, pipeline, options):
 
 use_fnapi = apiclient._use_fnapi(options)
 from apache_beam.transforms import environments
-self._default_environment = (
-environments.DockerEnvironment.from_container_image(
-apiclient.get_container_image_from_options(options),
-artifacts=environments.python_sdk_dependencies(options)))
+if options.view_as(DebugOptions).lookup_experiment(
+'prebuild_sdk_container'):

Review comment:
   make sense, moved to experiment.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [beam] y1chi commented on a change in pull request #12727: [BEAM-10844] Add experiment option prebuild_sdk_container to prebuild python sdk container with dependencies.

2020-09-18 Thread GitBox


y1chi commented on a change in pull request #12727:
URL: https://github.com/apache/beam/pull/12727#discussion_r491077972



##
File path: sdks/python/container/boot.go
##
@@ -283,3 +306,51 @@ func multiProcessExactlyOnce(actionFunc func(), 
completeFileName string) {
os.OpenFile(installCompleteFile, os.O_RDONLY|os.O_CREATE, 0666)
 
 }
+
+func processArtifactsInSetupOnlyMode() error {
+   if *artifacts == "" {
+   log.Fatal("No --artifacts provided along with --setup_only 
flag.")
+   }
+   workDir := filepath.Dir(*artifacts)
+   metadata, err := ioutil.ReadFile(*artifacts)
+   if err != nil {
+   log.Fatalf("Unable to open artifacts metadata file %v with 
error %v", *artifacts, err)
+   }
+   var infoJsons []string
+   if err := json.Unmarshal(metadata, ); err != nil {
+   log.Fatalf("Unable to parse metadata, error: %v", err)
+   }
+
+   files := make([]string, len(infoJsons))
+   for i, info := range infoJsons {
+   var artifactInformation pipepb.ArtifactInformation
+   if err := jsonpb.UnmarshalString(info, ); 
err != nil {
+   log.Fatalf("Unable to unmarshal artifact information 
from json string %v", info)
+   }
+
+   // For now we only expect artifacts in file type. The condition 
should be revisited if the assumption is not valid any more.
+   if artifactInformation.GetTypeUrn() != 
standardArtifactFileTypeUrn {
+   log.Fatalf("Expect file artifact type in setup only 
mode, found %v.", artifactInformation.GetTypeUrn())
+   }
+   filePayload := {}
+   if err := proto.Unmarshal(artifactInformation.GetTypePayload(), 
filePayload); err != nil {
+   log.Fatal("Unable to unmarshal artifact information 
type payload.")
+   }
+   if dir := filepath.Dir(filePayload.GetPath()); dir != workDir {
+   log.Fatalf("Artifact %v not stored in the same work 
directory %v of metadata file", filePayload.GetPath(), workDir)

Review comment:
   removed it. the installation script has an implicit requirements that 
all the artifacts are in same workdir. we can guarantee it while copying the 
artifacts.

##
File path: sdks/python/apache_beam/runners/portability/sdk_container_builder.py
##
@@ -0,0 +1,138 @@
+#
+# 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.
+#
+
+"""SdkContainerBuilder builds the portable SDK container with dependencies.
+
+It copies the right boot dependencies, namely: apache beam sdk, python packages
+from requirements.txt, python packages from extra_packages.txt, workflow
+tarball, into the latest public python sdk container image, and run the
+dependencies installation in advance with the boot program in setup only mode
+to build the new image.
+"""
+
+from __future__ import absolute_import
+
+import json
+import logging
+import os
+import shutil
+import subprocess
+import sys
+import tempfile
+import time
+import uuid
+
+from google.protobuf.json_format import MessageToJson
+
+from apache_beam.options.pipeline_options import SetupOptions
+from apache_beam.portability import common_urns
+from apache_beam.portability.api import beam_runner_api_pb2
+from apache_beam.runners.portability.stager import Stager
+
+ARTIFACTS_CONTAINER_DIR = '/opt/apache/beam/artifacts'
+ARTIFACTS_MANIFEST_FILE = 'artifacts_info.json'
+DOCKERFILE_TEMPLATE = (
+"""FROM apache/beam_python{major}.{minor}_sdk:latest
+RUN mkdir -p {workdir}
+COPY ./* {workdir}/
+RUN /opt/apache/beam/boot --setup_only --artifacts {workdir}/{manifest_file}

Review comment:
   done.

##
File path: sdks/python/container/boot.go
##
@@ -61,11 +66,22 @@ const (
sdkSrcFile= "dataflow_python_sdk.tar"
extraPackagesFile = "extra_packages.txt"
workerPoolIdEnv   = "BEAM_PYTHON_WORKER_POOL_ID"
+
+   // Setup result for the setup only mode.
+   setupResultFile = "/opt/apache/beam/setup_result.json"

Review comment:
   done.

##
File path: sdks/python/container/boot.go
##
@@ -203,13 +225,14 @@ func 

[GitHub] [beam] y1chi commented on a change in pull request #12727: [BEAM-10844] Add experiment option prebuild_sdk_container to prebuild python sdk container with dependencies.

2020-09-18 Thread GitBox


y1chi commented on a change in pull request #12727:
URL: https://github.com/apache/beam/pull/12727#discussion_r491077501



##
File path: sdks/python/container/boot.go
##
@@ -61,11 +66,22 @@ const (
sdkSrcFile= "dataflow_python_sdk.tar"
extraPackagesFile = "extra_packages.txt"
workerPoolIdEnv   = "BEAM_PYTHON_WORKER_POOL_ID"
+
+   // Setup result for the setup only mode.
+   setupResultFile = "/opt/apache/beam/setup_result.json"
+   standardArtifactFileTypeUrn = "beam:artifact:type:file:v1"

Review comment:
   I didn't find an already existing constant for this.





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org