Option #3 "external" would fit the Kubernetes use case we discussed a while
ago also. Container(s) can be part of the same pod and need to find the
runner.

There is another option: "embedded". When the SDK is Java and the runner
Flink (or all the other OSS runners), then harness can (optionally) run
embedded in the same JVM.

Thanks,
Thomas


On Thu, Aug 23, 2018 at 9:14 AM Henning Rohde <hero...@google.com> wrote:

> A process-based SDK harness does not IMO imply that the host is fully
> provisioned by the SDK/user and invoking the user command line in the
> context of the staged files is a critical aspect for it to work. So I
> consider staged artifact support needed. Also, I would like to suggest that
> we move to a concrete environment proto to crystalize what is actually
> being proposed. I'm not sure what activating a virtualenv would look like,
> for example. To start things off:
>
> message Environment {
>   string urn = 1;
>   bytes payload = 2;
> }
>
> // urn == "beam:env:docker:v1"
> message DockerPayload {
>   string container_image = 1;  // implicitly linux_amd64.
> }
>
> // urn == "beam:env:process:v1"
> message ProcessPayload {
>   string os = 1;  // "linux", "darwin", ..
>   string arch = 2;  // "amd64", ..
>   string command_line = 3;
> }
>
> // urn == "beam:env:external:v1"
> // (no payload)
>
> A runner may support any subset and reject any unsupported configuration.
> There are 3 kinds of environments that I think are useful:
>  (1) docker: works as currently. Offers the most flexibility for SDKs and
> users, especially when the runner is outside the control (such as hosted
> runners). The runner starts the SDK harnesses.
>  (2) process: as discussed here. The runner starts the SDK harnesses. The
> semantics is that the shell commandline is invoked in a directory rooted in
> the staged artifacts with the container contract arguments. It is up to the
> user and runner deployment to ensure that it makes sense, i.e., on windows
> a linux binary or bash script is not specified. Executing the user command
> in a shell env (bash, zsh, cmd, ..) ensures that paths and so on are set
> up:, i.e., specifying "java -jar foo" would actually work. Useful for cases
> where the user controls both the SDK and runner (such as locally) or when
> docker is not an option. Intended to be minimal and SDK/language agnostic.
>  (3) external: this is what I think Robert was alluding to. The runner
> does not start any SDK harnesses. Instead it waits for user-controlled SDK
> harnesses to connect. Useful for manually debugging SDK code (connect from
> code running in a debugger) or when the user code must run in a special or
> privileged environment. It's runner-specific how the SDK will need to
> connect.
>
> Part of the idea of placing this information in the environment is that
> pipelines can potentially use multiple, such as cross-windows/linux.
>
> Henning
>
> On Thu, Aug 23, 2018 at 6:44 AM Thomas Weise <t...@apache.org> wrote:
>
>> I would see support for staging libraries as optional / nice to have
>> since that can also be done as part of host provisioning (i.e. in the
>> Python case a virtual environment was already setup and just needs to be
>> activated).
>>
>> Depending on how the command that launches the harness is configured,
>> additional steps such as virtualenv activate or setting of other
>> environment variables can be included as well.
>>
>>
>> On Thu, Aug 23, 2018 at 5:15 AM Maximilian Michels <m...@apache.org>
>> wrote:
>>
>>> Just to recap:
>>>
>>>  From this and the other thread ("Bootstraping Beam's Job Server") we
>>> got sufficient evidence that process-based execution is a desired
>>> feature.
>>>
>>> Process-based execution as an alternative to dockerized execution
>>> https://issues.apache.org/jira/browse/BEAM-5187
>>>
>>> Which parts are executed as a process?
>>> => The SDK harness for user code
>>>
>>> What configuration options are supported?
>>> => Provide information about the target architecture (OS/CPU)
>>> => Staging libraries, as also supported by Docker
>>> => Activating a pre-existing environment (e.g. virutalenv)
>>>
>>>
>>> On 23.08.18 14:13, Maximilian Michels wrote:
>>> >> One thing to consider that we've talked about in the past. It might
>>> >> make sense to extend the environment proto and have the SDK be
>>> >> explicit about which kinds of environment it support
>>> >
>>> > +1 Encoding environment information there is a good idea.
>>> >
>>> >> Seems it will create a default docker url even if the
>>> >> hardness_docker_image is set to None in pipeline options. Shall we
>>> add
>>> >> another option or honor the None in this option to support the
>>> process
>>> >> job?
>>> >
>>> > Yes, if no Docker image is set the default one will be used. Currently
>>> > Docker is the only way to execute pipelines with the PortableRunner.
>>> If
>>> > the docker_image is not set, execution won't succeed.
>>> >
>>> > On 22.08.18 22:59, Xinyu Liu wrote:
>>> >> We are also interested in this Process JobBundleFactory as we are
>>> >> planning to fork a process to run python sdk in Samza runner, instead
>>> >> of using docker container. So this change will be helpful to us too.
>>> >> On the same note, we are trying out portable_runner.py to submit a
>>> >> python job. Seems it will create a default docker url even if the
>>> >> hardness_docker_image is set to None in pipeline options. Shall we
>>> add
>>> >> another option or honor the None in this option to support the
>>> process
>>> >> job? I made some local changes right now to walk around this.
>>> >>
>>> >> Thanks,
>>> >> Xinyu
>>> >>
>>> >> On Tue, Aug 21, 2018 at 12:25 PM, Henning Rohde <hero...@google.com
>>> >> <mailto:hero...@google.com>> wrote:
>>> >>
>>> >>     By "enum" in quotes, I meant the usual open URN style pattern not
>>> an
>>> >>     actual enum. Sorry if that wasn't clear.
>>> >>
>>> >>     On Tue, Aug 21, 2018 at 11:51 AM Lukasz Cwik <lc...@google.com
>>> >>     <mailto:lc...@google.com>> wrote:
>>> >>
>>> >>         I would model the environment to be more free form then enums
>>> >>         such that we have forward looking extensibility and would
>>> >>         suggest to follow the same pattern we use on PTransforms
>>> (using
>>> >>         an URN and a URN specific payload). Note that in this case we
>>> >>         may want to support a list of supported environments (e.g.
>>> java,
>>> >>         docker, python, ...).
>>> >>
>>> >>         On Tue, Aug 21, 2018 at 10:37 AM Henning Rohde
>>> >>         <hero...@google.com <mailto:hero...@google.com>> wrote:
>>> >>
>>> >>             One thing to consider that we've talked about in the past.
>>> >>             It might make sense to extend the environment proto and
>>> have
>>> >>             the SDK be explicit about which kinds of environment it
>>> >>             supports:
>>> >>
>>> >>
>>> >>
>>> https://github.com/apache/beam/blob/8c4f4babc0b0d55e7bddefa3f9f9ba65d21ef139/model/pipeline/src/main/proto/beam_runner_api.proto#L969
>>> >>
>>> >>
>>> >> <
>>> https://github.com/apache/beam/blob/8c4f4babc0b0d55e7bddefa3f9f9ba65d21ef139/model/pipeline/src/main/proto/beam_runner_api.proto#L969>
>>>
>>> >>
>>> >>
>>> >>             This choice might impact what files are staged or what
>>> not.
>>> >>             Some SDKs, such as Go, provide a compiled binary and
>>> _need_
>>> >>             to know what the target architecture is. Running on a mac
>>> >>             with and without docker, say, requires a different worker
>>> in
>>> >>             each case. If we add an "enum", we can also easily add the
>>> >>             external idea where the SDK/user starts the SDK harnesses
>>> >>             instead of the runner. Each runner may not support all
>>> types
>>> >>             of environments.
>>> >>
>>> >>             Henning
>>> >>
>>> >>             On Tue, Aug 21, 2018 at 2:52 AM Maximilian Michels
>>> >>             <m...@apache.org <mailto:m...@apache.org>> wrote:
>>> >>
>>> >>                 For reference, here is corresponding JIRA issue for
>>> this
>>> >>                 thread:
>>> >>                 https://issues.apache.org/jira/browse/BEAM-5187
>>> >>                 <https://issues.apache.org/jira/browse/BEAM-5187>
>>> >>
>>> >>                 On 16.08.18 11:15, Maximilian Michels wrote:
>>> >>                  > Makes sense to have an option to run the SDK
>>> harness
>>> >>                 in a non-dockerized
>>> >>                  > environment.
>>> >>                  >
>>> >>                  > I'm in the process of creating a Docker entry point
>>> >>                 for Flink's
>>> >>                  > JobServer[1]. I suppose you would also prefer to
>>> >>                 execute that one
>>> >>                  > standalone. We should make sure this is also an
>>> >> option.
>>> >>                  >
>>> >>                  > [1]
>>> https://issues.apache.org/jira/browse/BEAM-4130
>>> >>                 <https://issues.apache.org/jira/browse/BEAM-4130>
>>> >>                  >
>>> >>                  > On 16.08.18 07:42, Thomas Weise wrote:
>>> >>                  >> Yes, that's the proposal. Everything that would
>>> >>                 otherwise be packaged
>>> >>                  >> into the Docker container would need to be
>>> >>                 pre-installed in the host
>>> >>                  >> environment. In the case of Python SDK, that could
>>> >>                 simply mean a
>>> >>                  >> (frozen) virtual environment that was setup when
>>> the
>>> >>                 host was
>>> >>                  >> provisioned - the SDK harness process(es) will
>>> then
>>> >>                 just utilize that.
>>> >>                  >> Of course this flavor of SDK harness execution
>>> could
>>> >>                 also be useful in
>>> >>                  >> the local development environment, where right now
>>> >>                 someone who already
>>> >>                  >> has the Python environment needs to also install
>>> >>                 Docker and update a
>>> >>                  >> container to launch a Python SDK pipeline on the
>>> >>                 Flink runner.
>>> >>                  >>
>>> >>                  >> On Wed, Aug 15, 2018 at 12:40 PM Daniel Oliveira
>>> >>                 <danolive...@google.com <mailto:
>>> danolive...@google.com>
>>> >>                  >> <mailto:danolive...@google.com
>>> >>                 <mailto:danolive...@google.com>>> wrote:
>>> >>                  >>
>>> >>                  >>      I just want to clarify that I understand this
>>> >>                 correctly since I'm
>>> >>                  >>      not that familiar with the details behind all
>>> >>                 these execution
>>> >>                  >>      environments yet. Is the proposal to create a
>>> >>                 new JobBundleFactory
>>> >>                  >>      that instead of using Docker to create the
>>> >>                 environment that the new
>>> >>                  >>      processes will execute in, this
>>> >>                 JobBundleFactory would execute the
>>> >>                  >>      new processes directly in the host
>>> environment?
>>> >>                 So in practice if I
>>> >>                  >>      ran a pipeline with this JobBundleFactory the
>>> >>                 SDK Harness and Runner
>>> >>                  >>      Harness would both be executing directly on
>>> my
>>> >>                 machine and would
>>> >>                  >>      depend on me having the dependencies already
>>> >>                 present on my machine?
>>> >>                  >>
>>> >>                  >>      On Mon, Aug 13, 2018 at 5:50 PM Ankur Goenka
>>> >>                 <goe...@google.com <mailto:goe...@google.com>
>>> >>                  >>      <mailto:goe...@google.com
>>> >>                 <mailto:goe...@google.com>>> wrote:
>>> >>                  >>
>>> >>                  >>          Thanks for starting the discussion. I
>>> will
>>> >>                 be happy to help.
>>> >>                  >>          I agree, we should have pluggable
>>> >>                 SDKHarness environment Factory.
>>> >>                  >>          We can register multiple Environment
>>> >>                 factory using service
>>> >>                  >>          registry and use the PipelineOption to
>>> pick
>>> >>                 the right one on per
>>> >>                  >>          job basis.
>>> >>                  >>
>>> >>                  >>          There are a couple of things which are
>>> >>                 require to setup before
>>> >>                  >>          launching the process.
>>> >>                  >>
>>> >>                  >>            * Setting up the environment as done in
>>> >>                 boot.go [4]
>>> >>                  >>            * Retrieving and putting the artifacts
>>> in
>>> >>                 the right location.
>>> >>                  >>
>>> >>                  >>          You can probably leverage boot.go code to
>>> >>                 setup the environment.
>>> >>                  >>
>>> >>                  >>          Also, it will be useful to enumerate pros
>>> >>                 and cons of different
>>> >>                  >>          Environments to help users choose the
>>> right
>>> >>                 one.
>>> >>                  >>
>>> >>                  >>
>>> >>                  >>          On Mon, Aug 6, 2018 at 4:50 PM Thomas
>>> Weise
>>> >>                 <t...@apache.org <mailto:t...@apache.org>
>>> >>                  >>          <mailto:t...@apache.org
>>> >>                 <mailto:t...@apache.org>>> wrote:
>>> >>                  >>
>>> >>                  >>              Hi,
>>> >>                  >>
>>> >>                  >>              Currently the portable Flink runner
>>> >>                 only works with SDK
>>> >>                  >>              Docker containers for execution
>>> >>                 (DockerJobBundleFactory,
>>> >>                  >>              besides an in-process (embedded)
>>> >>                 factory option for testing
>>> >>                  >>              [1]). I'm considering adding another
>>> >>                 out of process
>>> >>                  >>              JobBundleFactory implementation that
>>> >>                 directly forks the
>>> >>                  >>              processes on the task manager host,
>>> >>                 eliminating the need for
>>> >>                  >>              Docker. This would work reasonably
>>> well
>>> >>                 in environments
>>> >>                  >>              where the dependencies (in this case
>>> >>                 Python) can easily be
>>> >>                  >>              tied into the host deployment (also
>>> >>                 within an application
>>> >>                  >>              specific Kubernetes pod).
>>> >>                  >>
>>> >>                  >>              There was already some discussion
>>> about
>>> >>                 alternative
>>> >>                  >>              JobBundleFactory implementation in
>>> [2].
>>> >>                 There is also a JIRA
>>> >>                  >>              to make the bundle factory pluggable
>>> >>                 [3], pending
>>> >>                  >>              availability of runner level options.
>>> >>                  >>
>>> >>                  >>              For a "ProcessBundleFactory", in
>>> >>                 addition to the Python
>>> >>                  >>              dependencies the environment would
>>> also
>>> >>                 need to have the Go
>>> >>                  >>              boot executable [4] (or a substitute
>>> >>                 thereof) to perform the
>>> >>                  >>              harness initialization.
>>> >>                  >>
>>> >>                  >>              Is anyone else interested in this SDK
>>> >>                 execution option or
>>> >>                  >>              has already investigated an
>>> alternative
>>> >>                 implementation?
>>> >>                  >>
>>> >>                  >>              Thanks,
>>> >>                  >>              Thomas
>>> >>                  >>
>>> >>                  >>              [1]
>>> >>                  >>
>>> >>
>>> >>
>>> https://github.com/apache/beam/blob/7958a379b0a37a89edc3a6ae4b5bc82fda41fcd6/runners/flink/src/test/java/org/apache/beam/runners/flink/PortableExecutionTest.java#L83
>>> >>
>>> >>
>>> >> <
>>> https://github.com/apache/beam/blob/7958a379b0a37a89edc3a6ae4b5bc82fda41fcd6/runners/flink/src/test/java/org/apache/beam/runners/flink/PortableExecutionTest.java#L83>
>>>
>>> >>
>>> >>                  >>
>>> >>                  >>              [2]
>>> >>                  >>
>>> >>
>>> >>
>>> https://lists.apache.org/thread.html/d6b6fde764796de31996db9bb5f9de3e7aaf0ab29b99d0adb52ac508@%3Cdev.beam.apache.org%3E
>>> >>
>>> >>
>>> >> <
>>> https://lists.apache.org/thread.html/d6b6fde764796de31996db9bb5f9de3e7aaf0ab29b99d0adb52ac508@%3Cdev.beam.apache.org%3E>
>>>
>>> >>
>>> >>                  >>
>>> >>                  >>              [3]
>>> >>                 https://issues.apache.org/jira/browse/BEAM-4819
>>> >>                 <https://issues.apache.org/jira/browse/BEAM-4819>
>>> >>                  >>
>>> >>                  >>              [4]
>>> >>
>>> >>
>>> https://github.com/apache/beam/blob/master/sdks/python/container/boot.go
>>> >>
>>> >> <
>>> https://github.com/apache/beam/blob/master/sdks/python/container/boot.go>
>>>
>>> >>
>>> >>                  >>
>>> >>
>>> >>                 --                 Max
>>> >>
>>> >>
>>> >
>>>
>>> --
>>> Max
>>>
>>

Reply via email to