Hi All,
If you have ever touched the docker topic in Flink, you probably noticed that we have multiple places in docs and repos which address its various concerns. We have prepared a FLIP [1] to simplify the perception of docker topic in Flink by users. It mostly advocates for an approach of extending official Flink image from the docker hub. For convenience, it can come with a set of bash utilities and documented examples of their usage. The utilities allow to: - run the docker image in various modes (single job, session master, task manager etc) - customise the extending Dockerfile - and its entry point Eventually, the FLIP suggests to remove all other user facing Dockerfiles and building scripts from Flink repo, move all docker docs to apache/flink-docker and adjust existing docker use cases to refer to this new approach (mostly Kubernetes now). The first contributed version of Flink docker integration also contained example and docs for the integration with Bluemix in IBM cloud. We also suggest to maintain it outside of Flink repository (cc Markus Müller). Thanks, Andrey [1] https://cwiki.apache.org/confluence/display/FLINK/FLIP-111%3A+Docker+image+unification |
Thanks for open this FLIP and summarize the current state of
Dockerfiles, Andrey! +1 for this idea. I have some minor comments / questions: - Regarding the flink_docker_utils#install_flink function, I think it should also support build from local dist and build from a user-defined archive. - It seems that the install_shaded_hadoop could be an option of install_flink. - Should we support JAVA 11? Currently, most of the docker file based on JAVA 8. - I do not understand how to set config options through "flink_docker_utils configure"? Does this step happen during the image build or the container start? If it happens during the image build, there would be a new image every time we change the config. If it just a part of the container entrypoint, I think there is no need to add a configure command, we could just add all dynamic config options to the args list of "start_jobmaster"/"start_session_jobmanager". Am I understanding this correctly? Best, Yangze Guo Best, Yangze Guo On Wed, Mar 4, 2020 at 5:34 PM Andrey Zagrebin <[hidden email]> wrote: > > Hi All, > > If you have ever touched the docker topic in Flink, you > probably noticed that we have multiple places in docs and repos which > address its various concerns. > > We have prepared a FLIP [1] to simplify the perception of docker topic in > Flink by users. It mostly advocates for an approach of extending official > Flink image from the docker hub. For convenience, it can come with a set of > bash utilities and documented examples of their usage. The utilities allow > to: > > - run the docker image in various modes (single job, session master, > task manager etc) > - customise the extending Dockerfile > - and its entry point > > Eventually, the FLIP suggests to remove all other user facing Dockerfiles > and building scripts from Flink repo, move all docker docs to > apache/flink-docker and adjust existing docker use cases to refer to this > new approach (mostly Kubernetes now). > > The first contributed version of Flink docker integration also contained > example and docs for the integration with Bluemix in IBM cloud. We also > suggest to maintain it outside of Flink repository (cc Markus Müller). > > Thanks, > Andrey > > [1] > https://cwiki.apache.org/confluence/display/FLINK/FLIP-111%3A+Docker+image+unification |
In reply to this post by Andrey Zagrebin-4
Hi Andrey,
Thanks for driving this significant FLIP. From the user ML, we could also know there are many users running Flink in container environment. Then the docker image will be the very basic requirement. Just as you say, we should provide a unified place for all various usage(e.g. session, job, native k8s, swarm, etc.). > About docker utils I really like the idea to provide some utils for the docker file and entry point. The `flink_docker_utils` will help to build the image easier. I am not sure about the `flink_docker_utils start_jobmaster`. Do you mean when we build a docker image, we need to add `RUN flink_docker_utils start_jobmaster` in the docker file? Why do we need this? > About docker entry point I agree with you that the docker entry point could more powerful with more functionality. Mostly, it is about to override the config options. If we support dynamic properties, i think it is more convenient for users without any learning curve. `docker run flink session_jobmanager -D rest.bind-port=8081` > About the logging Updating the `log4j-console.properties` to support multiple appender is a better option. Currently, the native K8s is suggesting users to debug the logs in this way[1]. However, there is also some problems. The stderr and stdout of JM/TM processes could not be forwarded to the docker container console. [1]. https://ci.apache.org/projects/flink/flink-docs-master/ops/deployment/native_kubernetes.html#log-files Best, Yang Andrey Zagrebin <[hidden email]> 于2020年3月4日周三 下午5:34写道: > Hi All, > > If you have ever touched the docker topic in Flink, you > probably noticed that we have multiple places in docs and repos which > address its various concerns. > > We have prepared a FLIP [1] to simplify the perception of docker topic in > Flink by users. It mostly advocates for an approach of extending official > Flink image from the docker hub. For convenience, it can come with a set of > bash utilities and documented examples of their usage. The utilities allow > to: > > - run the docker image in various modes (single job, session master, > task manager etc) > - customise the extending Dockerfile > - and its entry point > > Eventually, the FLIP suggests to remove all other user facing Dockerfiles > and building scripts from Flink repo, move all docker docs to > apache/flink-docker and adjust existing docker use cases to refer to this > new approach (mostly Kubernetes now). > > The first contributed version of Flink docker integration also contained > example and docs for the integration with Bluemix in IBM cloud. We also > suggest to maintain it outside of Flink repository (cc Markus Müller). > > Thanks, > Andrey > > [1] > > https://cwiki.apache.org/confluence/display/FLINK/FLIP-111%3A+Docker+image+unification > |
Hi Andrey,
thanks a lot for this proposal. The variety of Docker files in the project has been causing quite some confusion. For the entrypoint, have you considered to also allow setting configuration via environment variables as in "docker run -e FLINK_REST_BIN_PORT=8081 ..."? This is quite common and more flexible, e.g. it makes it very easy to pass values of Kubernetes Secrets into the Flink configuration. With respect to logging, I would opt to keep this very basic and to only support logging to the console (maybe with a fix for the web user interface). For everything else, users can easily build their own images based on library/flink (provide the dependencies, change the logging configuration). Cheers, Konstantin On Thu, Mar 5, 2020 at 11:01 AM Yang Wang <[hidden email]> wrote: > Hi Andrey, > > > Thanks for driving this significant FLIP. From the user ML, we could also > know there are > many users running Flink in container environment. Then the docker image > will be the > very basic requirement. Just as you say, we should provide a unified place > for all various > usage(e.g. session, job, native k8s, swarm, etc.). > > > > About docker utils > > I really like the idea to provide some utils for the docker file and entry > point. The > `flink_docker_utils` will help to build the image easier. I am not sure > about the > `flink_docker_utils start_jobmaster`. Do you mean when we build a docker > image, we > need to add `RUN flink_docker_utils start_jobmaster` in the docker file? > Why do we need this? > > > > About docker entry point > > I agree with you that the docker entry point could more powerful with more > functionality. > Mostly, it is about to override the config options. If we support dynamic > properties, i think > it is more convenient for users without any learning curve. > `docker run flink session_jobmanager -D rest.bind-port=8081` > > > > About the logging > > Updating the `log4j-console.properties` to support multiple appender is a > better option. > Currently, the native K8s is suggesting users to debug the logs in this > way[1]. However, > there is also some problems. The stderr and stdout of JM/TM processes > could not be > forwarded to the docker container console. > > > [1]. > https://ci.apache.org/projects/flink/flink-docs-master/ops/deployment/native_kubernetes.html#log-files > > > Best, > Yang > > > > > Andrey Zagrebin <[hidden email]> 于2020年3月4日周三 下午5:34写道: > >> Hi All, >> >> If you have ever touched the docker topic in Flink, you >> probably noticed that we have multiple places in docs and repos which >> address its various concerns. >> >> We have prepared a FLIP [1] to simplify the perception of docker topic in >> Flink by users. It mostly advocates for an approach of extending official >> Flink image from the docker hub. For convenience, it can come with a set >> of >> bash utilities and documented examples of their usage. The utilities allow >> to: >> >> - run the docker image in various modes (single job, session master, >> task manager etc) >> - customise the extending Dockerfile >> - and its entry point >> >> Eventually, the FLIP suggests to remove all other user facing Dockerfiles >> and building scripts from Flink repo, move all docker docs to >> apache/flink-docker and adjust existing docker use cases to refer to this >> new approach (mostly Kubernetes now). >> >> The first contributed version of Flink docker integration also contained >> example and docs for the integration with Bluemix in IBM cloud. We also >> suggest to maintain it outside of Flink repository (cc Markus Müller). >> >> Thanks, >> Andrey >> >> [1] >> >> https://cwiki.apache.org/confluence/display/FLINK/FLIP-111%3A+Docker+image+unification >> > -- Konstantin Knauf | Head of Product +49 160 91394525 Follow us @VervericaData Ververica <https://www.ververica.com/> -- Join Flink Forward <https://flink-forward.org/> - The Apache Flink Conference Stream Processing | Event Driven | Real Time -- Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany -- Ververica GmbH Registered at Amtsgericht Charlottenburg: HRB 158244 B Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji (Tony) Cheng |
Hi All,
Thanks a lot for the feedback! *@Yangze Guo* - Regarding the flink_docker_utils#install_flink function, I think it > should also support build from local dist and build from a > user-defined archive. I suppose you bring this up mostly for development purpose or powerful users. Most of normal users are usually interested in mainstream released versions of Flink. Although, you are bring a valid concern, my idea was to keep scope of this FLIP mostly for those normal users. The powerful users are usually capable to design a completely custom Dockerfile themselves. At the moment, we already have custom Dockerfiles e.g. for tests in flink-end-to-end-tests/test-scripts/docker-hadoop-secure-cluster/Dockerfile. We can add something similar for development purposes and maybe introduce a special maven goal. There is a maven docker plugin, afaik. I will add this to FLIP as next step. - It seems that the install_shaded_hadoop could be an option of > install_flink I woud rather think about this as a separate independent optional step. - Should we support JAVA 11? Currently, most of the docker file based on > JAVA 8. Indeed, it is a valid concern. Java version is a fundamental property of the docker image. To customise this in the current mainstream image is difficult, this would require to ship it w/o Java at all. Or this is a separate discussion whether we want to distribute docker hub images with different Java versions or just bump it to Java 11. This should be easy in a custom Dockerfile for development purposes though as mentioned before. - I do not understand how to set config options through "flink_docker_utils configure"? Does this step happen during the image > build or the container start? If it happens during the image build, > there would be a new image every time we change the config. If it just > a part of the container entrypoint, I think there is no need to add a > configure command, we could just add all dynamic config options to the > args list of "start_jobmaster"/"start_session_jobmanager". Am I > understanding this correctly? `flink_docker_utils configure ...` can be called everywhere: - while building a custom image (`RUN flink_docker_utils configure ..`) by extending our base image from docker hub (`from flink`) - in a custom entry point as well I will check this but if user can also pass a dynamic config option it also sounds like a good option Our standard entry point script in base image could just properly forward the arguments to the Flink process. @Yang Wang > About docker utils > I really like the idea to provide some utils for the docker file and entry > point. The > `flink_docker_utils` will help to build the image easier. I am not sure > about the > `flink_docker_utils start_jobmaster`. Do you mean when we build a docker > image, we > need to add `RUN flink_docker_utils start_jobmaster` in the docker file? > Why do we need this? This is a scripted action to start JM. It can be called everywhere. Indeed, it does not make too much sense to run it in Dockerfile. Mostly, the idea was to use in a custom entry point. When our base docker hub image is started its entry point can be also completely overridden. The actions are also sorted in the FLIP: for Dockerfile or for entry point. E.g. our standard entry point script in the base docker hub image can already use it. Anyways, it was just an example, the details are to be defined in Jira, imo. > About docker entry point > I agree with you that the docker entry point could more powerful with more > functionality. > Mostly, it is about to override the config options. If we support dynamic > properties, i think > it is more convenient for users without any learning curve. > `docker run flink session_jobmanager -D rest.bind-port=8081` Indeed, as mentioned before, it can be a better option. The standard entry point also decides at least what to run JM or TM. I think we will see what else makes sense to include there during the implementation. Some specifics may be more convenient to set with env vars as Konstantin mentioned. > About the logging > Updating the `log4j-console.properties` to support multiple appender is a > better option. > Currently, the native K8s is suggesting users to debug the logs in this > way[1]. However, > there is also some problems. The stderr and stdout of JM/TM processes could > not be > forwarded to the docker container console. Strange, we should check maybe there is a docker option to query the container's stderr output as well. If we forward Flink process stdout as usual in bash console, it should not be a problem. Why can it not be forwarded? @Konstantin Knauf For the entrypoint, have you considered to also allow setting configuration > via environment variables as in "docker run -e FLINK_REST_BIN_PORT=8081 > ..."? This is quite common and more flexible, e.g. it makes it very easy to > pass values of Kubernetes Secrets into the Flink configuration. This is indeed an interesting option to pass arguments to the entry point in general. For the config options, the dynamic args can be a better option as mentioned above. With respect to logging, I would opt to keep this very basic and to only > support logging to the console (maybe with a fix for the web user > interface). For everything else, users can easily build their own images > based on library/flink (provide the dependencies, change the logging > configuration). agree Thanks, Andrey On Sun, Mar 8, 2020 at 8:55 PM Konstantin Knauf <[hidden email]> wrote: > Hi Andrey, > > thanks a lot for this proposal. The variety of Docker files in the project > has been causing quite some confusion. > > For the entrypoint, have you considered to also allow setting > configuration via environment variables as in "docker run -e > FLINK_REST_BIN_PORT=8081 ..."? This is quite common and more flexible, e.g. > it makes it very easy to pass values of Kubernetes Secrets into the Flink > configuration. > > With respect to logging, I would opt to keep this very basic and to only > support logging to the console (maybe with a fix for the web user > interface). For everything else, users can easily build their own images > based on library/flink (provide the dependencies, change the logging > configuration). > > Cheers, > > Konstantin > > > On Thu, Mar 5, 2020 at 11:01 AM Yang Wang <[hidden email]> wrote: > >> Hi Andrey, >> >> >> Thanks for driving this significant FLIP. From the user ML, we could also >> know there are >> many users running Flink in container environment. Then the docker image >> will be the >> very basic requirement. Just as you say, we should provide a unified >> place for all various >> usage(e.g. session, job, native k8s, swarm, etc.). >> >> >> > About docker utils >> >> I really like the idea to provide some utils for the docker file and >> entry point. The >> `flink_docker_utils` will help to build the image easier. I am not sure >> about the >> `flink_docker_utils start_jobmaster`. Do you mean when we build a docker >> image, we >> need to add `RUN flink_docker_utils start_jobmaster` in the docker file? >> Why do we need this? >> >> >> > About docker entry point >> >> I agree with you that the docker entry point could more powerful with >> more functionality. >> Mostly, it is about to override the config options. If we support dynamic >> properties, i think >> it is more convenient for users without any learning curve. >> `docker run flink session_jobmanager -D rest.bind-port=8081` >> >> >> > About the logging >> >> Updating the `log4j-console.properties` to support multiple appender is a >> better option. >> Currently, the native K8s is suggesting users to debug the logs in this >> way[1]. However, >> there is also some problems. The stderr and stdout of JM/TM processes >> could not be >> forwarded to the docker container console. >> >> >> [1]. >> https://ci.apache.org/projects/flink/flink-docs-master/ops/deployment/native_kubernetes.html#log-files >> >> >> Best, >> Yang >> >> >> >> >> Andrey Zagrebin <[hidden email]> 于2020年3月4日周三 下午5:34写道: >> >>> Hi All, >>> >>> If you have ever touched the docker topic in Flink, you >>> probably noticed that we have multiple places in docs and repos which >>> address its various concerns. >>> >>> We have prepared a FLIP [1] to simplify the perception of docker topic in >>> Flink by users. It mostly advocates for an approach of extending official >>> Flink image from the docker hub. For convenience, it can come with a set >>> of >>> bash utilities and documented examples of their usage. The utilities >>> allow >>> to: >>> >>> - run the docker image in various modes (single job, session master, >>> task manager etc) >>> - customise the extending Dockerfile >>> - and its entry point >>> >>> Eventually, the FLIP suggests to remove all other user facing Dockerfiles >>> and building scripts from Flink repo, move all docker docs to >>> apache/flink-docker and adjust existing docker use cases to refer to this >>> new approach (mostly Kubernetes now). >>> >>> The first contributed version of Flink docker integration also contained >>> example and docs for the integration with Bluemix in IBM cloud. We also >>> suggest to maintain it outside of Flink repository (cc Markus Müller). >>> >>> Thanks, >>> Andrey >>> >>> [1] >>> >>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-111%3A+Docker+image+unification >>> >> > > -- > > Konstantin Knauf | Head of Product > > +49 160 91394525 > > > Follow us @VervericaData Ververica <https://www.ververica.com/> > > > -- > > Join Flink Forward <https://flink-forward.org/> - The Apache Flink > Conference > > Stream Processing | Event Driven | Real Time > > -- > > Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany > > -- > Ververica GmbH > Registered at Amtsgericht Charlottenburg: HRB 158244 B > Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji > (Tony) Cheng > |
Thanks for working on improvements to the Flink Docker container images.
This will be important as more and more users are looking to adopt Kubernetes and other deployment tooling that relies on Docker images. A generic, dynamic configuration mechanism based on environment variables is essential and it is already supported via envsubst and an environment variable that can supply a configuration fragment: https://github.com/apache/flink-docker/blob/09adf2dcd99abfb6180e1e2b5b917b288e0c01f6/docker-entrypoint.sh#L88 https://github.com/apache/flink-docker/blob/09adf2dcd99abfb6180e1e2b5b917b288e0c01f6/docker-entrypoint.sh#L85 This gives the necessary control for infrastructure use cases that aim to supply deployment tooling other users. An example in this category this is the FlinkK8sOperator: https://github.com/lyft/flinkk8soperator/tree/master/examples/wordcount On the flip side, attempting to support a fixed subset of configuration options is brittle and will probably lead to compatibility issues down the road: https://github.com/apache/flink-docker/blob/09adf2dcd99abfb6180e1e2b5b917b288e0c01f6/docker-entrypoint.sh#L97 Besides the configuration, it may be worthwhile to see in which other ways the base Docker images can provide more flexibility to incentivize wider adoption. I would second that it is desirable to support Java 11 and in general use a base image that allows the (straightforward) use of more recent versions of other software (Python etc.) https://github.com/apache/flink-docker/blob/d3416e720377e9b4c07a2d0f4591965264ac74c5/Dockerfile-debian.template#L19 Thanks, Thomas On Tue, Mar 10, 2020 at 12:26 PM Andrey Zagrebin <[hidden email]> wrote: > Hi All, > > Thanks a lot for the feedback! > > *@Yangze Guo* > > - Regarding the flink_docker_utils#install_flink function, I think it > > should also support build from local dist and build from a > > user-defined archive. > > I suppose you bring this up mostly for development purpose or powerful > users. > Most of normal users are usually interested in mainstream released versions > of Flink. > Although, you are bring a valid concern, my idea was to keep scope of this > FLIP mostly for those normal users. > The powerful users are usually capable to design a completely > custom Dockerfile themselves. > At the moment, we already have custom Dockerfiles e.g. for tests in > > flink-end-to-end-tests/test-scripts/docker-hadoop-secure-cluster/Dockerfile. > We can add something similar for development purposes and maybe introduce a > special maven goal. There is a maven docker plugin, afaik. > I will add this to FLIP as next step. > > - It seems that the install_shaded_hadoop could be an option of > > install_flink > > I woud rather think about this as a separate independent optional step. > > - Should we support JAVA 11? Currently, most of the docker file based on > > JAVA 8. > > Indeed, it is a valid concern. Java version is a fundamental property of > the docker image. > To customise this in the current mainstream image is difficult, this would > require to ship it w/o Java at all. > Or this is a separate discussion whether we want to distribute docker hub > images with different Java versions or just bump it to Java 11. > This should be easy in a custom Dockerfile for development purposes though > as mentioned before. > > - I do not understand how to set config options through > > "flink_docker_utils configure"? Does this step happen during the image > > build or the container start? If it happens during the image build, > > there would be a new image every time we change the config. If it just > > a part of the container entrypoint, I think there is no need to add a > > configure command, we could just add all dynamic config options to the > > args list of "start_jobmaster"/"start_session_jobmanager". Am I > > understanding this correctly? > > `flink_docker_utils configure ...` can be called everywhere: > - while building a custom image (`RUN flink_docker_utils configure ..`) by > extending our base image from docker hub (`from flink`) > - in a custom entry point as well > I will check this but if user can also pass a dynamic config option it also > sounds like a good option > Our standard entry point script in base image could just properly forward > the arguments to the Flink process. > > @Yang Wang > > > About docker utils > > I really like the idea to provide some utils for the docker file and > entry > > point. The > > `flink_docker_utils` will help to build the image easier. I am not sure > > about the > > `flink_docker_utils start_jobmaster`. Do you mean when we build a docker > > image, we > > need to add `RUN flink_docker_utils start_jobmaster` in the docker file? > > Why do we need this? > > This is a scripted action to start JM. It can be called everywhere. > Indeed, it does not make too much sense to run it in Dockerfile. > Mostly, the idea was to use in a custom entry point. When our base docker > hub image is started its entry point can be also completely overridden. > The actions are also sorted in the FLIP: for Dockerfile or for entry point. > E.g. our standard entry point script in the base docker hub image can > already use it. > Anyways, it was just an example, the details are to be defined in Jira, > imo. > > > About docker entry point > > I agree with you that the docker entry point could more powerful with > more > > functionality. > > Mostly, it is about to override the config options. If we support dynamic > > properties, i think > > it is more convenient for users without any learning curve. > > `docker run flink session_jobmanager -D rest.bind-port=8081` > > Indeed, as mentioned before, it can be a better option. > The standard entry point also decides at least what to run JM or TM. I > think we will see what else makes sense to include there during the > implementation. > Some specifics may be more convenient to set with env vars as Konstantin > mentioned. > > > About the logging > > Updating the `log4j-console.properties` to support multiple appender is a > > better option. > > Currently, the native K8s is suggesting users to debug the logs in this > > way[1]. However, > > there is also some problems. The stderr and stdout of JM/TM processes > could > > not be > > forwarded to the docker container console. > > Strange, we should check maybe there is a docker option to query the > container's stderr output as well. > If we forward Flink process stdout as usual in bash console, it should not > be a problem. Why can it not be forwarded? > > @Konstantin Knauf > > For the entrypoint, have you considered to also allow setting configuration > > via environment variables as in "docker run -e FLINK_REST_BIN_PORT=8081 > > ..."? This is quite common and more flexible, e.g. it makes it very easy > to > > pass values of Kubernetes Secrets into the Flink configuration. > > This is indeed an interesting option to pass arguments to the entry point > in general. > For the config options, the dynamic args can be a better option as > mentioned above. > > With respect to logging, I would opt to keep this very basic and to only > > support logging to the console (maybe with a fix for the web user > > interface). For everything else, users can easily build their own images > > based on library/flink (provide the dependencies, change the logging > > configuration). > > agree > > Thanks, > Andrey > > On Sun, Mar 8, 2020 at 8:55 PM Konstantin Knauf <[hidden email]> > wrote: > > > Hi Andrey, > > > > thanks a lot for this proposal. The variety of Docker files in the > project > > has been causing quite some confusion. > > > > For the entrypoint, have you considered to also allow setting > > configuration via environment variables as in "docker run -e > > FLINK_REST_BIN_PORT=8081 ..."? This is quite common and more flexible, > e.g. > > it makes it very easy to pass values of Kubernetes Secrets into the Flink > > configuration. > > > > With respect to logging, I would opt to keep this very basic and to only > > support logging to the console (maybe with a fix for the web user > > interface). For everything else, users can easily build their own images > > based on library/flink (provide the dependencies, change the logging > > configuration). > > > > Cheers, > > > > Konstantin > > > > > > On Thu, Mar 5, 2020 at 11:01 AM Yang Wang <[hidden email]> wrote: > > > >> Hi Andrey, > >> > >> > >> Thanks for driving this significant FLIP. From the user ML, we could > also > >> know there are > >> many users running Flink in container environment. Then the docker image > >> will be the > >> very basic requirement. Just as you say, we should provide a unified > >> place for all various > >> usage(e.g. session, job, native k8s, swarm, etc.). > >> > >> > >> > About docker utils > >> > >> I really like the idea to provide some utils for the docker file and > >> entry point. The > >> `flink_docker_utils` will help to build the image easier. I am not sure > >> about the > >> `flink_docker_utils start_jobmaster`. Do you mean when we build a docker > >> image, we > >> need to add `RUN flink_docker_utils start_jobmaster` in the docker file? > >> Why do we need this? > >> > >> > >> > About docker entry point > >> > >> I agree with you that the docker entry point could more powerful with > >> more functionality. > >> Mostly, it is about to override the config options. If we support > dynamic > >> properties, i think > >> it is more convenient for users without any learning curve. > >> `docker run flink session_jobmanager -D rest.bind-port=8081` > >> > >> > >> > About the logging > >> > >> Updating the `log4j-console.properties` to support multiple appender is > a > >> better option. > >> Currently, the native K8s is suggesting users to debug the logs in this > >> way[1]. However, > >> there is also some problems. The stderr and stdout of JM/TM processes > >> could not be > >> forwarded to the docker container console. > >> > >> > >> [1]. > >> > https://ci.apache.org/projects/flink/flink-docs-master/ops/deployment/native_kubernetes.html#log-files > >> > >> > >> Best, > >> Yang > >> > >> > >> > >> > >> Andrey Zagrebin <[hidden email]> 于2020年3月4日周三 下午5:34写道: > >> > >>> Hi All, > >>> > >>> If you have ever touched the docker topic in Flink, you > >>> probably noticed that we have multiple places in docs and repos which > >>> address its various concerns. > >>> > >>> We have prepared a FLIP [1] to simplify the perception of docker topic > in > >>> Flink by users. It mostly advocates for an approach of extending > official > >>> Flink image from the docker hub. For convenience, it can come with a > set > >>> of > >>> bash utilities and documented examples of their usage. The utilities > >>> allow > >>> to: > >>> > >>> - run the docker image in various modes (single job, session master, > >>> task manager etc) > >>> - customise the extending Dockerfile > >>> - and its entry point > >>> > >>> Eventually, the FLIP suggests to remove all other user facing > Dockerfiles > >>> and building scripts from Flink repo, move all docker docs to > >>> apache/flink-docker and adjust existing docker use cases to refer to > this > >>> new approach (mostly Kubernetes now). > >>> > >>> The first contributed version of Flink docker integration also > contained > >>> example and docs for the integration with Bluemix in IBM cloud. We also > >>> suggest to maintain it outside of Flink repository (cc Markus Müller). > >>> > >>> Thanks, > >>> Andrey > >>> > >>> [1] > >>> > >>> > https://cwiki.apache.org/confluence/display/FLINK/FLIP-111%3A+Docker+image+unification > >>> > >> > > > > -- > > > > Konstantin Knauf | Head of Product > > > > +49 160 91394525 > > > > > > Follow us @VervericaData Ververica <https://www.ververica.com/> > > > > > > -- > > > > Join Flink Forward <https://flink-forward.org/> - The Apache Flink > > Conference > > > > Stream Processing | Event Driven | Real Time > > > > -- > > > > Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany > > > > -- > > Ververica GmbH > > Registered at Amtsgericht Charlottenburg: HRB 158244 B > > Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji > > (Tony) Cheng > > > |
Thanks for the reply, Andrey.
Regarding building from local dist: - Yes, I bring this up mostly for development purpose. Since k8s is popular, I believe more and more developers would like to test their work on k8s cluster. I'm not sure should all developers write a custom docker file themselves in this scenario. Thus, I still prefer to provide a script for devs. - I agree to keep the scope of this FLIP mostly for those normal users. But as far as I can see, supporting building from local dist would not take much extra effort. - The maven docker plugin sounds good. I'll take a look at it. Regarding supporting JAVA 11: - Not sure if it is necessary to ship JAVA. Maybe we could just change the base image from openjdk:8-jre to openjdk:11-jre in template docker file[1]. Correct me if I understand incorrectly. Also, I agree to move this out of the scope of this FLIP if it indeed takes much extra effort. Regarding the custom configuration, the mechanism that Thomas mentioned LGTM. [1] https://github.com/apache/flink-docker/blob/master/Dockerfile-debian.template Best, Yangze Guo On Wed, Mar 11, 2020 at 5:52 AM Thomas Weise <[hidden email]> wrote: > > Thanks for working on improvements to the Flink Docker container images. This will be important as more and more users are looking to adopt Kubernetes and other deployment tooling that relies on Docker images. > > A generic, dynamic configuration mechanism based on environment variables is essential and it is already supported via envsubst and an environment variable that can supply a configuration fragment: > > https://github.com/apache/flink-docker/blob/09adf2dcd99abfb6180e1e2b5b917b288e0c01f6/docker-entrypoint.sh#L88 > https://github.com/apache/flink-docker/blob/09adf2dcd99abfb6180e1e2b5b917b288e0c01f6/docker-entrypoint.sh#L85 > > This gives the necessary control for infrastructure use cases that aim to supply deployment tooling other users. An example in this category this is the FlinkK8sOperator: > > https://github.com/lyft/flinkk8soperator/tree/master/examples/wordcount > > On the flip side, attempting to support a fixed subset of configuration options is brittle and will probably lead to compatibility issues down the road: > > https://github.com/apache/flink-docker/blob/09adf2dcd99abfb6180e1e2b5b917b288e0c01f6/docker-entrypoint.sh#L97 > > Besides the configuration, it may be worthwhile to see in which other ways the base Docker images can provide more flexibility to incentivize wider adoption. > > I would second that it is desirable to support Java 11 and in general use a base image that allows the (straightforward) use of more recent versions of other software (Python etc.) > > https://github.com/apache/flink-docker/blob/d3416e720377e9b4c07a2d0f4591965264ac74c5/Dockerfile-debian.template#L19 > > Thanks, > Thomas > > On Tue, Mar 10, 2020 at 12:26 PM Andrey Zagrebin <[hidden email]> wrote: >> >> Hi All, >> >> Thanks a lot for the feedback! >> >> *@Yangze Guo* >> >> - Regarding the flink_docker_utils#install_flink function, I think it >> > should also support build from local dist and build from a >> > user-defined archive. >> >> I suppose you bring this up mostly for development purpose or powerful >> users. >> Most of normal users are usually interested in mainstream released versions >> of Flink. >> Although, you are bring a valid concern, my idea was to keep scope of this >> FLIP mostly for those normal users. >> The powerful users are usually capable to design a completely >> custom Dockerfile themselves. >> At the moment, we already have custom Dockerfiles e.g. for tests in >> flink-end-to-end-tests/test-scripts/docker-hadoop-secure-cluster/Dockerfile. >> We can add something similar for development purposes and maybe introduce a >> special maven goal. There is a maven docker plugin, afaik. >> I will add this to FLIP as next step. >> >> - It seems that the install_shaded_hadoop could be an option of >> > install_flink >> >> I woud rather think about this as a separate independent optional step. >> >> - Should we support JAVA 11? Currently, most of the docker file based on >> > JAVA 8. >> >> Indeed, it is a valid concern. Java version is a fundamental property of >> the docker image. >> To customise this in the current mainstream image is difficult, this would >> require to ship it w/o Java at all. >> Or this is a separate discussion whether we want to distribute docker hub >> images with different Java versions or just bump it to Java 11. >> This should be easy in a custom Dockerfile for development purposes though >> as mentioned before. >> >> - I do not understand how to set config options through >> >> "flink_docker_utils configure"? Does this step happen during the image >> > build or the container start? If it happens during the image build, >> > there would be a new image every time we change the config. If it just >> > a part of the container entrypoint, I think there is no need to add a >> > configure command, we could just add all dynamic config options to the >> > args list of "start_jobmaster"/"start_session_jobmanager". Am I >> > understanding this correctly? >> >> `flink_docker_utils configure ...` can be called everywhere: >> - while building a custom image (`RUN flink_docker_utils configure ..`) by >> extending our base image from docker hub (`from flink`) >> - in a custom entry point as well >> I will check this but if user can also pass a dynamic config option it also >> sounds like a good option >> Our standard entry point script in base image could just properly forward >> the arguments to the Flink process. >> >> @Yang Wang >> >> > About docker utils >> > I really like the idea to provide some utils for the docker file and entry >> > point. The >> > `flink_docker_utils` will help to build the image easier. I am not sure >> > about the >> > `flink_docker_utils start_jobmaster`. Do you mean when we build a docker >> > image, we >> > need to add `RUN flink_docker_utils start_jobmaster` in the docker file? >> > Why do we need this? >> >> This is a scripted action to start JM. It can be called everywhere. >> Indeed, it does not make too much sense to run it in Dockerfile. >> Mostly, the idea was to use in a custom entry point. When our base docker >> hub image is started its entry point can be also completely overridden. >> The actions are also sorted in the FLIP: for Dockerfile or for entry point. >> E.g. our standard entry point script in the base docker hub image can >> already use it. >> Anyways, it was just an example, the details are to be defined in Jira, imo. >> >> > About docker entry point >> > I agree with you that the docker entry point could more powerful with more >> > functionality. >> > Mostly, it is about to override the config options. If we support dynamic >> > properties, i think >> > it is more convenient for users without any learning curve. >> > `docker run flink session_jobmanager -D rest.bind-port=8081` >> >> Indeed, as mentioned before, it can be a better option. >> The standard entry point also decides at least what to run JM or TM. I >> think we will see what else makes sense to include there during the >> implementation. >> Some specifics may be more convenient to set with env vars as Konstantin >> mentioned. >> >> > About the logging >> > Updating the `log4j-console.properties` to support multiple appender is a >> > better option. >> > Currently, the native K8s is suggesting users to debug the logs in this >> > way[1]. However, >> > there is also some problems. The stderr and stdout of JM/TM processes could >> > not be >> > forwarded to the docker container console. >> >> Strange, we should check maybe there is a docker option to query the >> container's stderr output as well. >> If we forward Flink process stdout as usual in bash console, it should not >> be a problem. Why can it not be forwarded? >> >> @Konstantin Knauf >> >> For the entrypoint, have you considered to also allow setting configuration >> > via environment variables as in "docker run -e FLINK_REST_BIN_PORT=8081 >> > ..."? This is quite common and more flexible, e.g. it makes it very easy to >> > pass values of Kubernetes Secrets into the Flink configuration. >> >> This is indeed an interesting option to pass arguments to the entry point >> in general. >> For the config options, the dynamic args can be a better option as >> mentioned above. >> >> With respect to logging, I would opt to keep this very basic and to only >> > support logging to the console (maybe with a fix for the web user >> > interface). For everything else, users can easily build their own images >> > based on library/flink (provide the dependencies, change the logging >> > configuration). >> >> agree >> >> Thanks, >> Andrey >> >> On Sun, Mar 8, 2020 at 8:55 PM Konstantin Knauf <[hidden email]> >> wrote: >> >> > Hi Andrey, >> > >> > thanks a lot for this proposal. The variety of Docker files in the project >> > has been causing quite some confusion. >> > >> > For the entrypoint, have you considered to also allow setting >> > configuration via environment variables as in "docker run -e >> > FLINK_REST_BIN_PORT=8081 ..."? This is quite common and more flexible, e.g. >> > it makes it very easy to pass values of Kubernetes Secrets into the Flink >> > configuration. >> > >> > With respect to logging, I would opt to keep this very basic and to only >> > support logging to the console (maybe with a fix for the web user >> > interface). For everything else, users can easily build their own images >> > based on library/flink (provide the dependencies, change the logging >> > configuration). >> > >> > Cheers, >> > >> > Konstantin >> > >> > >> > On Thu, Mar 5, 2020 at 11:01 AM Yang Wang <[hidden email]> wrote: >> > >> >> Hi Andrey, >> >> >> >> >> >> Thanks for driving this significant FLIP. From the user ML, we could also >> >> know there are >> >> many users running Flink in container environment. Then the docker image >> >> will be the >> >> very basic requirement. Just as you say, we should provide a unified >> >> place for all various >> >> usage(e.g. session, job, native k8s, swarm, etc.). >> >> >> >> >> >> > About docker utils >> >> >> >> I really like the idea to provide some utils for the docker file and >> >> entry point. The >> >> `flink_docker_utils` will help to build the image easier. I am not sure >> >> about the >> >> `flink_docker_utils start_jobmaster`. Do you mean when we build a docker >> >> image, we >> >> need to add `RUN flink_docker_utils start_jobmaster` in the docker file? >> >> Why do we need this? >> >> >> >> >> >> > About docker entry point >> >> >> >> I agree with you that the docker entry point could more powerful with >> >> more functionality. >> >> Mostly, it is about to override the config options. If we support dynamic >> >> properties, i think >> >> it is more convenient for users without any learning curve. >> >> `docker run flink session_jobmanager -D rest.bind-port=8081` >> >> >> >> >> >> > About the logging >> >> >> >> Updating the `log4j-console.properties` to support multiple appender is a >> >> better option. >> >> Currently, the native K8s is suggesting users to debug the logs in this >> >> way[1]. However, >> >> there is also some problems. The stderr and stdout of JM/TM processes >> >> could not be >> >> forwarded to the docker container console. >> >> >> >> >> >> [1]. >> >> https://ci.apache.org/projects/flink/flink-docs-master/ops/deployment/native_kubernetes.html#log-files >> >> >> >> >> >> Best, >> >> Yang >> >> >> >> >> >> >> >> >> >> Andrey Zagrebin <[hidden email]> 于2020年3月4日周三 下午5:34写道: >> >> >> >>> Hi All, >> >>> >> >>> If you have ever touched the docker topic in Flink, you >> >>> probably noticed that we have multiple places in docs and repos which >> >>> address its various concerns. >> >>> >> >>> We have prepared a FLIP [1] to simplify the perception of docker topic in >> >>> Flink by users. It mostly advocates for an approach of extending official >> >>> Flink image from the docker hub. For convenience, it can come with a set >> >>> of >> >>> bash utilities and documented examples of their usage. The utilities >> >>> allow >> >>> to: >> >>> >> >>> - run the docker image in various modes (single job, session master, >> >>> task manager etc) >> >>> - customise the extending Dockerfile >> >>> - and its entry point >> >>> >> >>> Eventually, the FLIP suggests to remove all other user facing Dockerfiles >> >>> and building scripts from Flink repo, move all docker docs to >> >>> apache/flink-docker and adjust existing docker use cases to refer to this >> >>> new approach (mostly Kubernetes now). >> >>> >> >>> The first contributed version of Flink docker integration also contained >> >>> example and docs for the integration with Bluemix in IBM cloud. We also >> >>> suggest to maintain it outside of Flink repository (cc Markus Müller). >> >>> >> >>> Thanks, >> >>> Andrey >> >>> >> >>> [1] >> >>> >> >>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-111%3A+Docker+image+unification >> >>> >> >> >> > >> > -- >> > >> > Konstantin Knauf | Head of Product >> > >> > +49 160 91394525 >> > >> > >> > Follow us @VervericaData Ververica <https://www.ververica.com/> >> > >> > >> > -- >> > >> > Join Flink Forward <https://flink-forward.org/> - The Apache Flink >> > Conference >> > >> > Stream Processing | Event Driven | Real Time >> > >> > -- >> > >> > Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany >> > >> > -- >> > Ververica GmbH >> > Registered at Amtsgericht Charlottenburg: HRB 158244 B >> > Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji >> > (Tony) Cheng >> > |
Thanks for the further feedback Thomas and Yangze.
> A generic, dynamic configuration mechanism based on environment variables is essential and it is already supported via envsubst and an environment variable that can supply a configuration fragment True, we already have this. As I understand this was introduced for flexibility to template a custom flink-conf.yaml with env vars, put it into the FLINK_PROPERTIES and merge it with the default one. Could we achieve the same with the dynamic properties (-Drpc.port=1234), passed as image args to run it, instead of FLINK_PROPERTIES? They could be also parametrised with env vars. This would require jobmanager.sh to properly propagate them to the StandaloneSessionClusterEntrypoint though: https://github.com/docker-flink/docker-flink/pull/82#issuecomment-525285552 cc @Till This would provide a unified configuration approach. > On the flip side, attempting to support a fixed subset of configuration options is brittle and will probably lead to compatibility issues down the road I agree with it. The idea was to have just some shortcut scripted functions to set options in flink-conf.yaml for a custom Dockerfile or entry point script. TASK_MANAGER_NUMBER_OF_TASK_SLOTS could be set as a dynamic property of started JM. I am not sure how many users depend on it. Maybe we could remove it. It also looks we already have somewhat unclean state in the docker-entrypoint.sh where some ports are set the hardcoded values and then FLINK_PROPERTIES are applied potentially duplicating options in the result flink-conf.yaml. I can see some potential usage of env vars as standard entry point args but for purposes related to something which cannot be achieved by passing entry point args, like changing flink-conf.yaml options. Nothing comes into my mind at the moment. It could be some setting specific to the running mode of the entry point. The mode itself can stay the first arg of the entry point. > I would second that it is desirable to support Java 11 > Regarding supporting JAVA 11: > - Not sure if it is necessary to ship JAVA. Maybe we could just change > the base image from openjdk:8-jre to openjdk:11-jre in template docker > file[1]. Correct me if I understand incorrectly. Also, I agree to move > this out of the scope of this FLIP if it indeed takes much extra > effort. This is what I meant by bumping up the Java version in the docker hub Flink image: FROM openjdk:8-jre -> FROM openjdk:11-jre This can be polled dependently in user mailing list. > and in general use a base image that allows the (straightforward) use of more recent versions of other software (Python etc.) This can be polled whether to always include some version of python into the docker hub image. A potential problem here is once it is there, it is some hassle to remove/change it in a custom extended Dockerfile. It would be also nice to avoid maintaining images for various combinations of installed Java/Scala/Python in docker hub. > Regarding building from local dist: > - Yes, I bring this up mostly for development purpose. Since k8s is > popular, I believe more and more developers would like to test their > work on k8s cluster. I'm not sure should all developers write a custom > docker file themselves in this scenario. Thus, I still prefer to > provide a script for devs. > - I agree to keep the scope of this FLIP mostly for those normal > users. But as far as I can see, supporting building from local dist > would not take much extra effort. > - The maven docker plugin sounds good. I'll take a look at it. I would see any scripts introduced in this FLIP also as potential building blocks for a custom dev Dockerfile. Maybe, this will be all what we need for dev images or we write a dev Dockerfile, highly parametrised for building a dev image. If scripts stay in apache/flink-docker, it is also somewhat inconvenient to use them in the main Flink repo but possible. If we move them to apache/flink then we will have to e.g. include them into the release to make them easily available in apache/flink-docker and maintain them in main repo, although they are only docker specific. All in all, I would say, once we implement them, we can revisit this topic. Best, Andrey On Wed, Mar 11, 2020 at 8:58 AM Yangze Guo <[hidden email]> wrote: > Thanks for the reply, Andrey. > > Regarding building from local dist: > - Yes, I bring this up mostly for development purpose. Since k8s is > popular, I believe more and more developers would like to test their > work on k8s cluster. I'm not sure should all developers write a custom > docker file themselves in this scenario. Thus, I still prefer to > provide a script for devs. > - I agree to keep the scope of this FLIP mostly for those normal > users. But as far as I can see, supporting building from local dist > would not take much extra effort. > - The maven docker plugin sounds good. I'll take a look at it. > > Regarding supporting JAVA 11: > - Not sure if it is necessary to ship JAVA. Maybe we could just change > the base image from openjdk:8-jre to openjdk:11-jre in template docker > file[1]. Correct me if I understand incorrectly. Also, I agree to move > this out of the scope of this FLIP if it indeed takes much extra > effort. > > Regarding the custom configuration, the mechanism that Thomas mentioned > LGTM. > > [1] > https://github.com/apache/flink-docker/blob/master/Dockerfile-debian.template > > Best, > Yangze Guo > > On Wed, Mar 11, 2020 at 5:52 AM Thomas Weise <[hidden email]> wrote: > > > > Thanks for working on improvements to the Flink Docker container images. > This will be important as more and more users are looking to adopt > Kubernetes and other deployment tooling that relies on Docker images. > > > > A generic, dynamic configuration mechanism based on environment > variables is essential and it is already supported via envsubst and an > environment variable that can supply a configuration fragment: > > > > > https://github.com/apache/flink-docker/blob/09adf2dcd99abfb6180e1e2b5b917b288e0c01f6/docker-entrypoint.sh#L88 > > > https://github.com/apache/flink-docker/blob/09adf2dcd99abfb6180e1e2b5b917b288e0c01f6/docker-entrypoint.sh#L85 > > > > This gives the necessary control for infrastructure use cases that aim > to supply deployment tooling other users. An example in this category this > is the FlinkK8sOperator: > > > > https://github.com/lyft/flinkk8soperator/tree/master/examples/wordcount > > > > On the flip side, attempting to support a fixed subset of configuration > options is brittle and will probably lead to compatibility issues down the > road: > > > > > https://github.com/apache/flink-docker/blob/09adf2dcd99abfb6180e1e2b5b917b288e0c01f6/docker-entrypoint.sh#L97 > > > > Besides the configuration, it may be worthwhile to see in which other > ways the base Docker images can provide more flexibility to incentivize > wider adoption. > > > > I would second that it is desirable to support Java 11 and in general > use a base image that allows the (straightforward) use of more recent > versions of other software (Python etc.) > > > > > https://github.com/apache/flink-docker/blob/d3416e720377e9b4c07a2d0f4591965264ac74c5/Dockerfile-debian.template#L19 > > > > Thanks, > > Thomas > > > > On Tue, Mar 10, 2020 at 12:26 PM Andrey Zagrebin <[hidden email]> > wrote: > >> > >> Hi All, > >> > >> Thanks a lot for the feedback! > >> > >> *@Yangze Guo* > >> > >> - Regarding the flink_docker_utils#install_flink function, I think it > >> > should also support build from local dist and build from a > >> > user-defined archive. > >> > >> I suppose you bring this up mostly for development purpose or powerful > >> users. > >> Most of normal users are usually interested in mainstream released > versions > >> of Flink. > >> Although, you are bring a valid concern, my idea was to keep scope of > this > >> FLIP mostly for those normal users. > >> The powerful users are usually capable to design a completely > >> custom Dockerfile themselves. > >> At the moment, we already have custom Dockerfiles e.g. for tests in > >> > flink-end-to-end-tests/test-scripts/docker-hadoop-secure-cluster/Dockerfile. > >> We can add something similar for development purposes and maybe > introduce a > >> special maven goal. There is a maven docker plugin, afaik. > >> I will add this to FLIP as next step. > >> > >> - It seems that the install_shaded_hadoop could be an option of > >> > install_flink > >> > >> I woud rather think about this as a separate independent optional step. > >> > >> - Should we support JAVA 11? Currently, most of the docker file based on > >> > JAVA 8. > >> > >> Indeed, it is a valid concern. Java version is a fundamental property of > >> the docker image. > >> To customise this in the current mainstream image is difficult, this > would > >> require to ship it w/o Java at all. > >> Or this is a separate discussion whether we want to distribute docker > hub > >> images with different Java versions or just bump it to Java 11. > >> This should be easy in a custom Dockerfile for development purposes > though > >> as mentioned before. > >> > >> - I do not understand how to set config options through > >> > >> "flink_docker_utils configure"? Does this step happen during the image > >> > build or the container start? If it happens during the image build, > >> > there would be a new image every time we change the config. If it just > >> > a part of the container entrypoint, I think there is no need to add a > >> > configure command, we could just add all dynamic config options to the > >> > args list of "start_jobmaster"/"start_session_jobmanager". Am I > >> > understanding this correctly? > >> > >> `flink_docker_utils configure ...` can be called everywhere: > >> - while building a custom image (`RUN flink_docker_utils configure ..`) > by > >> extending our base image from docker hub (`from flink`) > >> - in a custom entry point as well > >> I will check this but if user can also pass a dynamic config option it > also > >> sounds like a good option > >> Our standard entry point script in base image could just properly > forward > >> the arguments to the Flink process. > >> > >> @Yang Wang > >> > >> > About docker utils > >> > I really like the idea to provide some utils for the docker file and > entry > >> > point. The > >> > `flink_docker_utils` will help to build the image easier. I am not > sure > >> > about the > >> > `flink_docker_utils start_jobmaster`. Do you mean when we build a > docker > >> > image, we > >> > need to add `RUN flink_docker_utils start_jobmaster` in the docker > file? > >> > Why do we need this? > >> > >> This is a scripted action to start JM. It can be called everywhere. > >> Indeed, it does not make too much sense to run it in Dockerfile. > >> Mostly, the idea was to use in a custom entry point. When our base > docker > >> hub image is started its entry point can be also completely overridden. > >> The actions are also sorted in the FLIP: for Dockerfile or for entry > point. > >> E.g. our standard entry point script in the base docker hub image can > >> already use it. > >> Anyways, it was just an example, the details are to be defined in Jira, > imo. > >> > >> > About docker entry point > >> > I agree with you that the docker entry point could more powerful with > more > >> > functionality. > >> > Mostly, it is about to override the config options. If we support > dynamic > >> > properties, i think > >> > it is more convenient for users without any learning curve. > >> > `docker run flink session_jobmanager -D rest.bind-port=8081` > >> > >> Indeed, as mentioned before, it can be a better option. > >> The standard entry point also decides at least what to run JM or TM. I > >> think we will see what else makes sense to include there during the > >> implementation. > >> Some specifics may be more convenient to set with env vars as Konstantin > >> mentioned. > >> > >> > About the logging > >> > Updating the `log4j-console.properties` to support multiple appender > is a > >> > better option. > >> > Currently, the native K8s is suggesting users to debug the logs in > this > >> > way[1]. However, > >> > there is also some problems. The stderr and stdout of JM/TM processes > could > >> > not be > >> > forwarded to the docker container console. > >> > >> Strange, we should check maybe there is a docker option to query the > >> container's stderr output as well. > >> If we forward Flink process stdout as usual in bash console, it should > not > >> be a problem. Why can it not be forwarded? > >> > >> @Konstantin Knauf > >> > >> For the entrypoint, have you considered to also allow setting > configuration > >> > via environment variables as in "docker run -e > FLINK_REST_BIN_PORT=8081 > >> > ..."? This is quite common and more flexible, e.g. it makes it very > easy to > >> > pass values of Kubernetes Secrets into the Flink configuration. > >> > >> This is indeed an interesting option to pass arguments to the entry > point > >> in general. > >> For the config options, the dynamic args can be a better option as > >> mentioned above. > >> > >> With respect to logging, I would opt to keep this very basic and to only > >> > support logging to the console (maybe with a fix for the web user > >> > interface). For everything else, users can easily build their own > images > >> > based on library/flink (provide the dependencies, change the logging > >> > configuration). > >> > >> agree > >> > >> Thanks, > >> Andrey > >> > >> On Sun, Mar 8, 2020 at 8:55 PM Konstantin Knauf < > [hidden email]> > >> wrote: > >> > >> > Hi Andrey, > >> > > >> > thanks a lot for this proposal. The variety of Docker files in the > project > >> > has been causing quite some confusion. > >> > > >> > For the entrypoint, have you considered to also allow setting > >> > configuration via environment variables as in "docker run -e > >> > FLINK_REST_BIN_PORT=8081 ..."? This is quite common and more > flexible, e.g. > >> > it makes it very easy to pass values of Kubernetes Secrets into the > Flink > >> > configuration. > >> > > >> > With respect to logging, I would opt to keep this very basic and to > only > >> > support logging to the console (maybe with a fix for the web user > >> > interface). For everything else, users can easily build their own > images > >> > based on library/flink (provide the dependencies, change the logging > >> > configuration). > >> > > >> > Cheers, > >> > > >> > Konstantin > >> > > >> > > >> > On Thu, Mar 5, 2020 at 11:01 AM Yang Wang <[hidden email]> > wrote: > >> > > >> >> Hi Andrey, > >> >> > >> >> > >> >> Thanks for driving this significant FLIP. From the user ML, we could > also > >> >> know there are > >> >> many users running Flink in container environment. Then the docker > image > >> >> will be the > >> >> very basic requirement. Just as you say, we should provide a unified > >> >> place for all various > >> >> usage(e.g. session, job, native k8s, swarm, etc.). > >> >> > >> >> > >> >> > About docker utils > >> >> > >> >> I really like the idea to provide some utils for the docker file and > >> >> entry point. The > >> >> `flink_docker_utils` will help to build the image easier. I am not > sure > >> >> about the > >> >> `flink_docker_utils start_jobmaster`. Do you mean when we build a > docker > >> >> image, we > >> >> need to add `RUN flink_docker_utils start_jobmaster` in the docker > file? > >> >> Why do we need this? > >> >> > >> >> > >> >> > About docker entry point > >> >> > >> >> I agree with you that the docker entry point could more powerful with > >> >> more functionality. > >> >> Mostly, it is about to override the config options. If we support > dynamic > >> >> properties, i think > >> >> it is more convenient for users without any learning curve. > >> >> `docker run flink session_jobmanager -D rest.bind-port=8081` > >> >> > >> >> > >> >> > About the logging > >> >> > >> >> Updating the `log4j-console.properties` to support multiple appender > is a > >> >> better option. > >> >> Currently, the native K8s is suggesting users to debug the logs in > this > >> >> way[1]. However, > >> >> there is also some problems. The stderr and stdout of JM/TM processes > >> >> could not be > >> >> forwarded to the docker container console. > >> >> > >> >> > >> >> [1]. > >> >> > https://ci.apache.org/projects/flink/flink-docs-master/ops/deployment/native_kubernetes.html#log-files > >> >> > >> >> > >> >> Best, > >> >> Yang > >> >> > >> >> > >> >> > >> >> > >> >> Andrey Zagrebin <[hidden email]> 于2020年3月4日周三 下午5:34写道: > >> >> > >> >>> Hi All, > >> >>> > >> >>> If you have ever touched the docker topic in Flink, you > >> >>> probably noticed that we have multiple places in docs and repos > which > >> >>> address its various concerns. > >> >>> > >> >>> We have prepared a FLIP [1] to simplify the perception of docker > topic in > >> >>> Flink by users. It mostly advocates for an approach of extending > official > >> >>> Flink image from the docker hub. For convenience, it can come with > a set > >> >>> of > >> >>> bash utilities and documented examples of their usage. The utilities > >> >>> allow > >> >>> to: > >> >>> > >> >>> - run the docker image in various modes (single job, session > master, > >> >>> task manager etc) > >> >>> - customise the extending Dockerfile > >> >>> - and its entry point > >> >>> > >> >>> Eventually, the FLIP suggests to remove all other user facing > Dockerfiles > >> >>> and building scripts from Flink repo, move all docker docs to > >> >>> apache/flink-docker and adjust existing docker use cases to refer > to this > >> >>> new approach (mostly Kubernetes now). > >> >>> > >> >>> The first contributed version of Flink docker integration also > contained > >> >>> example and docs for the integration with Bluemix in IBM cloud. We > also > >> >>> suggest to maintain it outside of Flink repository (cc Markus > Müller). > >> >>> > >> >>> Thanks, > >> >>> Andrey > >> >>> > >> >>> [1] > >> >>> > >> >>> > https://cwiki.apache.org/confluence/display/FLINK/FLIP-111%3A+Docker+image+unification > >> >>> > >> >> > >> > > >> > -- > >> > > >> > Konstantin Knauf | Head of Product > >> > > >> > +49 160 91394525 > >> > > >> > > >> > Follow us @VervericaData Ververica <https://www.ververica.com/> > >> > > >> > > >> > -- > >> > > >> > Join Flink Forward <https://flink-forward.org/> - The Apache Flink > >> > Conference > >> > > >> > Stream Processing | Event Driven | Real Time > >> > > >> > -- > >> > > >> > Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany > >> > > >> > -- > >> > Ververica GmbH > >> > Registered at Amtsgericht Charlottenburg: HRB 158244 B > >> > Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, > Ji > >> > (Tony) Cheng > >> > > |
Hi Andrey,
Thanks for your explanation. > About the logging What i mean is we could not forward the stdout/stderr to local files and docker stdout at the same time by using log4j. For the jobmanager.log/taskmanager.log, it works quite well since we only need to add a console appender in the log4j.properties. I am just curious how to forward the stdout/stderr to local files and docker stdout at the same time by using log4j :) Best, Yang Andrey Zagrebin <[hidden email]> 于2020年3月16日周一 下午4:58写道: > Thanks for the further feedback Thomas and Yangze. > > > A generic, dynamic configuration mechanism based on environment > variables is essential and it is already supported via envsubst and an > environment variable that can supply a configuration fragment > > True, we already have this. As I understand this was introduced for > flexibility to template a custom flink-conf.yaml with env vars, put it into > the FLINK_PROPERTIES and merge it with the default one. > Could we achieve the same with the dynamic properties (-Drpc.port=1234), > passed as image args to run it, instead of FLINK_PROPERTIES? > They could be also parametrised with env vars. This would require > jobmanager.sh to properly propagate them to > the StandaloneSessionClusterEntrypoint though: > https://github.com/docker-flink/docker-flink/pull/82#issuecomment-525285552 > cc @Till > This would provide a unified configuration approach. > > > On the flip side, attempting to support a fixed subset of configuration > options is brittle and will probably lead to compatibility issues down the > road > > I agree with it. The idea was to have just some shortcut scripted > functions to set options in flink-conf.yaml for a custom Dockerfile or > entry point script. > TASK_MANAGER_NUMBER_OF_TASK_SLOTS could be set as a dynamic property of > started JM. > I am not sure how many users depend on it. Maybe we could remove it. > It also looks we already have somewhat unclean state in > the docker-entrypoint.sh where some ports are set the hardcoded values > and then FLINK_PROPERTIES are applied potentially duplicating options in > the result flink-conf.yaml. > > I can see some potential usage of env vars as standard entry point args > but for purposes related to something which cannot be achieved by passing > entry point args, like changing flink-conf.yaml options. Nothing comes into > my mind at the moment. It could be some setting specific to the running > mode of the entry point. The mode itself can stay the first arg of the > entry point. > > > I would second that it is desirable to support Java 11 > > > Regarding supporting JAVA 11: > > - Not sure if it is necessary to ship JAVA. Maybe we could just change > > the base image from openjdk:8-jre to openjdk:11-jre in template docker > > file[1]. Correct me if I understand incorrectly. Also, I agree to move > > this out of the scope of this FLIP if it indeed takes much extra > > effort. > > This is what I meant by bumping up the Java version in the docker hub > Flink image: > FROM openjdk:8-jre -> FROM openjdk:11-jre > This can be polled dependently in user mailing list. > > > and in general use a base image that allows the (straightforward) use of > more recent versions of other software (Python etc.) > > This can be polled whether to always include some version of python into > the docker hub image. > A potential problem here is once it is there, it is some hassle to > remove/change it in a custom extended Dockerfile. > > It would be also nice to avoid maintaining images for various combinations > of installed Java/Scala/Python in docker hub. > > > Regarding building from local dist: > > - Yes, I bring this up mostly for development purpose. Since k8s is > > popular, I believe more and more developers would like to test their > > work on k8s cluster. I'm not sure should all developers write a custom > > docker file themselves in this scenario. Thus, I still prefer to > > provide a script for devs. > > - I agree to keep the scope of this FLIP mostly for those normal > > users. But as far as I can see, supporting building from local dist > > would not take much extra effort. > > - The maven docker plugin sounds good. I'll take a look at it. > > I would see any scripts introduced in this FLIP also as potential building > blocks for a custom dev Dockerfile. > Maybe, this will be all what we need for dev images or we write a dev > Dockerfile, highly parametrised for building a dev image. > If scripts stay in apache/flink-docker, it is also somewhat inconvenient > to use them in the main Flink repo but possible. > If we move them to apache/flink then we will have to e.g. include them > into the release to make them easily available in apache/flink-docker and > maintain them in main repo, although they are only docker specific. > All in all, I would say, once we implement them, we can revisit this topic. > > Best, > Andrey > > On Wed, Mar 11, 2020 at 8:58 AM Yangze Guo <[hidden email]> wrote: > >> Thanks for the reply, Andrey. >> >> Regarding building from local dist: >> - Yes, I bring this up mostly for development purpose. Since k8s is >> popular, I believe more and more developers would like to test their >> work on k8s cluster. I'm not sure should all developers write a custom >> docker file themselves in this scenario. Thus, I still prefer to >> provide a script for devs. >> - I agree to keep the scope of this FLIP mostly for those normal >> users. But as far as I can see, supporting building from local dist >> would not take much extra effort. >> - The maven docker plugin sounds good. I'll take a look at it. >> >> Regarding supporting JAVA 11: >> - Not sure if it is necessary to ship JAVA. Maybe we could just change >> the base image from openjdk:8-jre to openjdk:11-jre in template docker >> file[1]. Correct me if I understand incorrectly. Also, I agree to move >> this out of the scope of this FLIP if it indeed takes much extra >> effort. >> >> Regarding the custom configuration, the mechanism that Thomas mentioned >> LGTM. >> >> [1] >> https://github.com/apache/flink-docker/blob/master/Dockerfile-debian.template >> >> Best, >> Yangze Guo >> >> On Wed, Mar 11, 2020 at 5:52 AM Thomas Weise <[hidden email]> wrote: >> > >> > Thanks for working on improvements to the Flink Docker container >> images. This will be important as more and more users are looking to adopt >> Kubernetes and other deployment tooling that relies on Docker images. >> > >> > A generic, dynamic configuration mechanism based on environment >> variables is essential and it is already supported via envsubst and an >> environment variable that can supply a configuration fragment: >> > >> > >> https://github.com/apache/flink-docker/blob/09adf2dcd99abfb6180e1e2b5b917b288e0c01f6/docker-entrypoint.sh#L88 >> > >> https://github.com/apache/flink-docker/blob/09adf2dcd99abfb6180e1e2b5b917b288e0c01f6/docker-entrypoint.sh#L85 >> > >> > This gives the necessary control for infrastructure use cases that aim >> to supply deployment tooling other users. An example in this category this >> is the FlinkK8sOperator: >> > >> > https://github.com/lyft/flinkk8soperator/tree/master/examples/wordcount >> > >> > On the flip side, attempting to support a fixed subset of configuration >> options is brittle and will probably lead to compatibility issues down the >> road: >> > >> > >> https://github.com/apache/flink-docker/blob/09adf2dcd99abfb6180e1e2b5b917b288e0c01f6/docker-entrypoint.sh#L97 >> > >> > Besides the configuration, it may be worthwhile to see in which other >> ways the base Docker images can provide more flexibility to incentivize >> wider adoption. >> > >> > I would second that it is desirable to support Java 11 and in general >> use a base image that allows the (straightforward) use of more recent >> versions of other software (Python etc.) >> > >> > >> https://github.com/apache/flink-docker/blob/d3416e720377e9b4c07a2d0f4591965264ac74c5/Dockerfile-debian.template#L19 >> > >> > Thanks, >> > Thomas >> > >> > On Tue, Mar 10, 2020 at 12:26 PM Andrey Zagrebin <[hidden email]> >> wrote: >> >> >> >> Hi All, >> >> >> >> Thanks a lot for the feedback! >> >> >> >> *@Yangze Guo* >> >> >> >> - Regarding the flink_docker_utils#install_flink function, I think it >> >> > should also support build from local dist and build from a >> >> > user-defined archive. >> >> >> >> I suppose you bring this up mostly for development purpose or powerful >> >> users. >> >> Most of normal users are usually interested in mainstream released >> versions >> >> of Flink. >> >> Although, you are bring a valid concern, my idea was to keep scope of >> this >> >> FLIP mostly for those normal users. >> >> The powerful users are usually capable to design a completely >> >> custom Dockerfile themselves. >> >> At the moment, we already have custom Dockerfiles e.g. for tests in >> >> >> flink-end-to-end-tests/test-scripts/docker-hadoop-secure-cluster/Dockerfile. >> >> We can add something similar for development purposes and maybe >> introduce a >> >> special maven goal. There is a maven docker plugin, afaik. >> >> I will add this to FLIP as next step. >> >> >> >> - It seems that the install_shaded_hadoop could be an option of >> >> > install_flink >> >> >> >> I woud rather think about this as a separate independent optional step. >> >> >> >> - Should we support JAVA 11? Currently, most of the docker file based >> on >> >> > JAVA 8. >> >> >> >> Indeed, it is a valid concern. Java version is a fundamental property >> of >> >> the docker image. >> >> To customise this in the current mainstream image is difficult, this >> would >> >> require to ship it w/o Java at all. >> >> Or this is a separate discussion whether we want to distribute docker >> hub >> >> images with different Java versions or just bump it to Java 11. >> >> This should be easy in a custom Dockerfile for development purposes >> though >> >> as mentioned before. >> >> >> >> - I do not understand how to set config options through >> >> >> >> "flink_docker_utils configure"? Does this step happen during the image >> >> > build or the container start? If it happens during the image build, >> >> > there would be a new image every time we change the config. If it >> just >> >> > a part of the container entrypoint, I think there is no need to add a >> >> > configure command, we could just add all dynamic config options to >> the >> >> > args list of "start_jobmaster"/"start_session_jobmanager". Am I >> >> > understanding this correctly? >> >> >> >> `flink_docker_utils configure ...` can be called everywhere: >> >> - while building a custom image (`RUN flink_docker_utils configure >> ..`) by >> >> extending our base image from docker hub (`from flink`) >> >> - in a custom entry point as well >> >> I will check this but if user can also pass a dynamic config option it >> also >> >> sounds like a good option >> >> Our standard entry point script in base image could just properly >> forward >> >> the arguments to the Flink process. >> >> >> >> @Yang Wang >> >> >> >> > About docker utils >> >> > I really like the idea to provide some utils for the docker file and >> entry >> >> > point. The >> >> > `flink_docker_utils` will help to build the image easier. I am not >> sure >> >> > about the >> >> > `flink_docker_utils start_jobmaster`. Do you mean when we build a >> docker >> >> > image, we >> >> > need to add `RUN flink_docker_utils start_jobmaster` in the docker >> file? >> >> > Why do we need this? >> >> >> >> This is a scripted action to start JM. It can be called everywhere. >> >> Indeed, it does not make too much sense to run it in Dockerfile. >> >> Mostly, the idea was to use in a custom entry point. When our base >> docker >> >> hub image is started its entry point can be also completely overridden. >> >> The actions are also sorted in the FLIP: for Dockerfile or for entry >> point. >> >> E.g. our standard entry point script in the base docker hub image can >> >> already use it. >> >> Anyways, it was just an example, the details are to be defined in >> Jira, imo. >> >> >> >> > About docker entry point >> >> > I agree with you that the docker entry point could more powerful >> with more >> >> > functionality. >> >> > Mostly, it is about to override the config options. If we support >> dynamic >> >> > properties, i think >> >> > it is more convenient for users without any learning curve. >> >> > `docker run flink session_jobmanager -D rest.bind-port=8081` >> >> >> >> Indeed, as mentioned before, it can be a better option. >> >> The standard entry point also decides at least what to run JM or TM. I >> >> think we will see what else makes sense to include there during the >> >> implementation. >> >> Some specifics may be more convenient to set with env vars as >> Konstantin >> >> mentioned. >> >> >> >> > About the logging >> >> > Updating the `log4j-console.properties` to support multiple appender >> is a >> >> > better option. >> >> > Currently, the native K8s is suggesting users to debug the logs in >> this >> >> > way[1]. However, >> >> > there is also some problems. The stderr and stdout of JM/TM >> processes could >> >> > not be >> >> > forwarded to the docker container console. >> >> >> >> Strange, we should check maybe there is a docker option to query the >> >> container's stderr output as well. >> >> If we forward Flink process stdout as usual in bash console, it should >> not >> >> be a problem. Why can it not be forwarded? >> >> >> >> @Konstantin Knauf >> >> >> >> For the entrypoint, have you considered to also allow setting >> configuration >> >> > via environment variables as in "docker run -e >> FLINK_REST_BIN_PORT=8081 >> >> > ..."? This is quite common and more flexible, e.g. it makes it very >> easy to >> >> > pass values of Kubernetes Secrets into the Flink configuration. >> >> >> >> This is indeed an interesting option to pass arguments to the entry >> point >> >> in general. >> >> For the config options, the dynamic args can be a better option as >> >> mentioned above. >> >> >> >> With respect to logging, I would opt to keep this very basic and to >> only >> >> > support logging to the console (maybe with a fix for the web user >> >> > interface). For everything else, users can easily build their own >> images >> >> > based on library/flink (provide the dependencies, change the logging >> >> > configuration). >> >> >> >> agree >> >> >> >> Thanks, >> >> Andrey >> >> >> >> On Sun, Mar 8, 2020 at 8:55 PM Konstantin Knauf < >> [hidden email]> >> >> wrote: >> >> >> >> > Hi Andrey, >> >> > >> >> > thanks a lot for this proposal. The variety of Docker files in the >> project >> >> > has been causing quite some confusion. >> >> > >> >> > For the entrypoint, have you considered to also allow setting >> >> > configuration via environment variables as in "docker run -e >> >> > FLINK_REST_BIN_PORT=8081 ..."? This is quite common and more >> flexible, e.g. >> >> > it makes it very easy to pass values of Kubernetes Secrets into the >> Flink >> >> > configuration. >> >> > >> >> > With respect to logging, I would opt to keep this very basic and to >> only >> >> > support logging to the console (maybe with a fix for the web user >> >> > interface). For everything else, users can easily build their own >> images >> >> > based on library/flink (provide the dependencies, change the logging >> >> > configuration). >> >> > >> >> > Cheers, >> >> > >> >> > Konstantin >> >> > >> >> > >> >> > On Thu, Mar 5, 2020 at 11:01 AM Yang Wang <[hidden email]> >> wrote: >> >> > >> >> >> Hi Andrey, >> >> >> >> >> >> >> >> >> Thanks for driving this significant FLIP. From the user ML, we >> could also >> >> >> know there are >> >> >> many users running Flink in container environment. Then the docker >> image >> >> >> will be the >> >> >> very basic requirement. Just as you say, we should provide a unified >> >> >> place for all various >> >> >> usage(e.g. session, job, native k8s, swarm, etc.). >> >> >> >> >> >> >> >> >> > About docker utils >> >> >> >> >> >> I really like the idea to provide some utils for the docker file and >> >> >> entry point. The >> >> >> `flink_docker_utils` will help to build the image easier. I am not >> sure >> >> >> about the >> >> >> `flink_docker_utils start_jobmaster`. Do you mean when we build a >> docker >> >> >> image, we >> >> >> need to add `RUN flink_docker_utils start_jobmaster` in the docker >> file? >> >> >> Why do we need this? >> >> >> >> >> >> >> >> >> > About docker entry point >> >> >> >> >> >> I agree with you that the docker entry point could more powerful >> with >> >> >> more functionality. >> >> >> Mostly, it is about to override the config options. If we support >> dynamic >> >> >> properties, i think >> >> >> it is more convenient for users without any learning curve. >> >> >> `docker run flink session_jobmanager -D rest.bind-port=8081` >> >> >> >> >> >> >> >> >> > About the logging >> >> >> >> >> >> Updating the `log4j-console.properties` to support multiple >> appender is a >> >> >> better option. >> >> >> Currently, the native K8s is suggesting users to debug the logs in >> this >> >> >> way[1]. However, >> >> >> there is also some problems. The stderr and stdout of JM/TM >> processes >> >> >> could not be >> >> >> forwarded to the docker container console. >> >> >> >> >> >> >> >> >> [1]. >> >> >> >> https://ci.apache.org/projects/flink/flink-docs-master/ops/deployment/native_kubernetes.html#log-files >> >> >> >> >> >> >> >> >> Best, >> >> >> Yang >> >> >> >> >> >> >> >> >> >> >> >> >> >> >> Andrey Zagrebin <[hidden email]> 于2020年3月4日周三 下午5:34写道: >> >> >> >> >> >>> Hi All, >> >> >>> >> >> >>> If you have ever touched the docker topic in Flink, you >> >> >>> probably noticed that we have multiple places in docs and repos >> which >> >> >>> address its various concerns. >> >> >>> >> >> >>> We have prepared a FLIP [1] to simplify the perception of docker >> topic in >> >> >>> Flink by users. It mostly advocates for an approach of extending >> official >> >> >>> Flink image from the docker hub. For convenience, it can come with >> a set >> >> >>> of >> >> >>> bash utilities and documented examples of their usage. The >> utilities >> >> >>> allow >> >> >>> to: >> >> >>> >> >> >>> - run the docker image in various modes (single job, session >> master, >> >> >>> task manager etc) >> >> >>> - customise the extending Dockerfile >> >> >>> - and its entry point >> >> >>> >> >> >>> Eventually, the FLIP suggests to remove all other user facing >> Dockerfiles >> >> >>> and building scripts from Flink repo, move all docker docs to >> >> >>> apache/flink-docker and adjust existing docker use cases to refer >> to this >> >> >>> new approach (mostly Kubernetes now). >> >> >>> >> >> >>> The first contributed version of Flink docker integration also >> contained >> >> >>> example and docs for the integration with Bluemix in IBM cloud. We >> also >> >> >>> suggest to maintain it outside of Flink repository (cc Markus >> Müller). >> >> >>> >> >> >>> Thanks, >> >> >>> Andrey >> >> >>> >> >> >>> [1] >> >> >>> >> >> >>> >> https://cwiki.apache.org/confluence/display/FLINK/FLIP-111%3A+Docker+image+unification >> >> >>> >> >> >> >> >> > >> >> > -- >> >> > >> >> > Konstantin Knauf | Head of Product >> >> > >> >> > +49 160 91394525 >> >> > >> >> > >> >> > Follow us @VervericaData Ververica <https://www.ververica.com/> >> >> > >> >> > >> >> > -- >> >> > >> >> > Join Flink Forward <https://flink-forward.org/> - The Apache Flink >> >> > Conference >> >> > >> >> > Stream Processing | Event Driven | Real Time >> >> > >> >> > -- >> >> > >> >> > Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany >> >> > >> >> > -- >> >> > Ververica GmbH >> >> > Registered at Amtsgericht Charlottenburg: HRB 158244 B >> >> > Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, >> Ji >> >> > (Tony) Cheng >> >> > >> > |
In reply to this post by Andrey Zagrebin-4
-->
On Mon, Mar 16, 2020 at 1:58 AM Andrey Zagrebin <[hidden email]> wrote: > Thanks for the further feedback Thomas and Yangze. > > > A generic, dynamic configuration mechanism based on environment variables > is essential and it is already supported via envsubst and an environment > variable that can supply a configuration fragment > > True, we already have this. As I understand this was introduced for > flexibility to template a custom flink-conf.yaml with env vars, put it into > the FLINK_PROPERTIES and merge it with the default one. > Could we achieve the same with the dynamic properties (-Drpc.port=1234), > passed as image args to run it, instead of FLINK_PROPERTIES? > They could be also parametrised with env vars. This would require > jobmanager.sh to properly propagate them to > the StandaloneSessionClusterEntrypoint though: > https://github.com/docker-flink/docker-flink/pull/82#issuecomment-525285552 > cc @Till > This would provide a unified configuration approach. > > need to generate the -Dabc .. -Dxyz entry point command instead of setting the FLINK_PROPERTIES environment variable? Potentially that introduces additional complexity for little gain. Do most deployment platforms that support Docker containers handle the command line route well? Backward compatibility may also be a concern. > > On the flip side, attempting to support a fixed subset of configuration > options is brittle and will probably lead to compatibility issues down the > road > > I agree with it. The idea was to have just some shortcut scripted functions > to set options in flink-conf.yaml for a custom Dockerfile or entry point > script. > TASK_MANAGER_NUMBER_OF_TASK_SLOTS could be set as a dynamic property of > started JM. > I am not sure how many users depend on it. Maybe we could remove it. > It also looks we already have somewhat unclean state in > the docker-entrypoint.sh where some ports are set the hardcoded values > and then FLINK_PROPERTIES are applied potentially duplicating options in > the result flink-conf.yaml. > That is indeed possible and duplicate entries from FLINK_PROPERTIES prevail. Unfortunately, the special cases you mention were already established and the generic mechanism was added later for the k8s operators. > > I can see some potential usage of env vars as standard entry point args but > for purposes related to something which cannot be achieved by passing entry > point args, like changing flink-conf.yaml options. Nothing comes into my > mind at the moment. It could be some setting specific to the running mode > of the entry point. The mode itself can stay the first arg of the entry > point. > > > I would second that it is desirable to support Java 11 > > > Regarding supporting JAVA 11: > > - Not sure if it is necessary to ship JAVA. Maybe we could just change > > the base image from openjdk:8-jre to openjdk:11-jre in template docker > > file[1]. Correct me if I understand incorrectly. Also, I agree to move > > this out of the scope of this FLIP if it indeed takes much extra > > effort. > > This is what I meant by bumping up the Java version in the docker hub Flink > image: > FROM openjdk:8-jre -> FROM openjdk:11-jre > This can be polled dependently in user mailing list. > That sounds reasonable as long as we can still support both Java versions (i.e. provide separate images for 8 and 11). > > > and in general use a base image that allows the (straightforward) use of > more recent versions of other software (Python etc.) > > This can be polled whether to always include some version of python into > the docker hub image. > A potential problem here is once it is there, it is some hassle to > remove/change it in a custom extended Dockerfile. > > It would be also nice to avoid maintaining images for various combinations > of installed Java/Scala/Python in docker hub. > > > Regarding building from local dist: > > - Yes, I bring this up mostly for development purpose. Since k8s is > > popular, I believe more and more developers would like to test their > > work on k8s cluster. I'm not sure should all developers write a custom > > docker file themselves in this scenario. Thus, I still prefer to > > provide a script for devs. > > - I agree to keep the scope of this FLIP mostly for those normal > > users. But as far as I can see, supporting building from local dist > > would not take much extra effort. > > - The maven docker plugin sounds good. I'll take a look at it. > > I would see any scripts introduced in this FLIP also as potential building > blocks for a custom dev Dockerfile. > Maybe, this will be all what we need for dev images or we write a dev > Dockerfile, highly parametrised for building a dev image. > If scripts stay in apache/flink-docker, it is also somewhat inconvenient to > use them in the main Flink repo but possible. > If we move them to apache/flink then we will have to e.g. include them into > the release to make them easily available in apache/flink-docker and > maintain them in main repo, although they are only docker specific. > All in all, I would say, once we implement them, we can revisit this topic. > > Best, > Andrey > > On Wed, Mar 11, 2020 at 8:58 AM Yangze Guo <[hidden email]> wrote: > > > Thanks for the reply, Andrey. > > > > Regarding building from local dist: > > - Yes, I bring this up mostly for development purpose. Since k8s is > > popular, I believe more and more developers would like to test their > > work on k8s cluster. I'm not sure should all developers write a custom > > docker file themselves in this scenario. Thus, I still prefer to > > provide a script for devs. > > - I agree to keep the scope of this FLIP mostly for those normal > > users. But as far as I can see, supporting building from local dist > > would not take much extra effort. > > - The maven docker plugin sounds good. I'll take a look at it. > > > > Regarding supporting JAVA 11: > > - Not sure if it is necessary to ship JAVA. Maybe we could just change > > the base image from openjdk:8-jre to openjdk:11-jre in template docker > > file[1]. Correct me if I understand incorrectly. Also, I agree to move > > this out of the scope of this FLIP if it indeed takes much extra > > effort. > > > > Regarding the custom configuration, the mechanism that Thomas mentioned > > LGTM. > > > > [1] > > > https://github.com/apache/flink-docker/blob/master/Dockerfile-debian.template > > > > Best, > > Yangze Guo > > > > On Wed, Mar 11, 2020 at 5:52 AM Thomas Weise <[hidden email]> wrote: > > > > > > Thanks for working on improvements to the Flink Docker container > images. > > This will be important as more and more users are looking to adopt > > Kubernetes and other deployment tooling that relies on Docker images. > > > > > > A generic, dynamic configuration mechanism based on environment > > variables is essential and it is already supported via envsubst and an > > environment variable that can supply a configuration fragment: > > > > > > > > > https://github.com/apache/flink-docker/blob/09adf2dcd99abfb6180e1e2b5b917b288e0c01f6/docker-entrypoint.sh#L88 > > > > > > https://github.com/apache/flink-docker/blob/09adf2dcd99abfb6180e1e2b5b917b288e0c01f6/docker-entrypoint.sh#L85 > > > > > > This gives the necessary control for infrastructure use cases that aim > > to supply deployment tooling other users. An example in this category > this > > is the FlinkK8sOperator: > > > > > > > https://github.com/lyft/flinkk8soperator/tree/master/examples/wordcount > > > > > > On the flip side, attempting to support a fixed subset of configuration > > options is brittle and will probably lead to compatibility issues down > the > > road: > > > > > > > > > https://github.com/apache/flink-docker/blob/09adf2dcd99abfb6180e1e2b5b917b288e0c01f6/docker-entrypoint.sh#L97 > > > > > > Besides the configuration, it may be worthwhile to see in which other > > ways the base Docker images can provide more flexibility to incentivize > > wider adoption. > > > > > > I would second that it is desirable to support Java 11 and in general > > use a base image that allows the (straightforward) use of more recent > > versions of other software (Python etc.) > > > > > > > > > https://github.com/apache/flink-docker/blob/d3416e720377e9b4c07a2d0f4591965264ac74c5/Dockerfile-debian.template#L19 > > > > > > Thanks, > > > Thomas > > > > > > On Tue, Mar 10, 2020 at 12:26 PM Andrey Zagrebin <[hidden email] > > > > wrote: > > >> > > >> Hi All, > > >> > > >> Thanks a lot for the feedback! > > >> > > >> *@Yangze Guo* > > >> > > >> - Regarding the flink_docker_utils#install_flink function, I think it > > >> > should also support build from local dist and build from a > > >> > user-defined archive. > > >> > > >> I suppose you bring this up mostly for development purpose or powerful > > >> users. > > >> Most of normal users are usually interested in mainstream released > > versions > > >> of Flink. > > >> Although, you are bring a valid concern, my idea was to keep scope of > > this > > >> FLIP mostly for those normal users. > > >> The powerful users are usually capable to design a completely > > >> custom Dockerfile themselves. > > >> At the moment, we already have custom Dockerfiles e.g. for tests in > > >> > > > flink-end-to-end-tests/test-scripts/docker-hadoop-secure-cluster/Dockerfile. > > >> We can add something similar for development purposes and maybe > > introduce a > > >> special maven goal. There is a maven docker plugin, afaik. > > >> I will add this to FLIP as next step. > > >> > > >> - It seems that the install_shaded_hadoop could be an option of > > >> > install_flink > > >> > > >> I woud rather think about this as a separate independent optional > step. > > >> > > >> - Should we support JAVA 11? Currently, most of the docker file based > on > > >> > JAVA 8. > > >> > > >> Indeed, it is a valid concern. Java version is a fundamental property > of > > >> the docker image. > > >> To customise this in the current mainstream image is difficult, this > > would > > >> require to ship it w/o Java at all. > > >> Or this is a separate discussion whether we want to distribute docker > > hub > > >> images with different Java versions or just bump it to Java 11. > > >> This should be easy in a custom Dockerfile for development purposes > > though > > >> as mentioned before. > > >> > > >> - I do not understand how to set config options through > > >> > > >> "flink_docker_utils configure"? Does this step happen during the image > > >> > build or the container start? If it happens during the image build, > > >> > there would be a new image every time we change the config. If it > just > > >> > a part of the container entrypoint, I think there is no need to add > a > > >> > configure command, we could just add all dynamic config options to > the > > >> > args list of "start_jobmaster"/"start_session_jobmanager". Am I > > >> > understanding this correctly? > > >> > > >> `flink_docker_utils configure ...` can be called everywhere: > > >> - while building a custom image (`RUN flink_docker_utils configure > ..`) > > by > > >> extending our base image from docker hub (`from flink`) > > >> - in a custom entry point as well > > >> I will check this but if user can also pass a dynamic config option it > > also > > >> sounds like a good option > > >> Our standard entry point script in base image could just properly > > forward > > >> the arguments to the Flink process. > > >> > > >> @Yang Wang > > >> > > >> > About docker utils > > >> > I really like the idea to provide some utils for the docker file and > > entry > > >> > point. The > > >> > `flink_docker_utils` will help to build the image easier. I am not > > sure > > >> > about the > > >> > `flink_docker_utils start_jobmaster`. Do you mean when we build a > > docker > > >> > image, we > > >> > need to add `RUN flink_docker_utils start_jobmaster` in the docker > > file? > > >> > Why do we need this? > > >> > > >> This is a scripted action to start JM. It can be called everywhere. > > >> Indeed, it does not make too much sense to run it in Dockerfile. > > >> Mostly, the idea was to use in a custom entry point. When our base > > docker > > >> hub image is started its entry point can be also completely > overridden. > > >> The actions are also sorted in the FLIP: for Dockerfile or for entry > > point. > > >> E.g. our standard entry point script in the base docker hub image can > > >> already use it. > > >> Anyways, it was just an example, the details are to be defined in > Jira, > > imo. > > >> > > >> > About docker entry point > > >> > I agree with you that the docker entry point could more powerful > with > > more > > >> > functionality. > > >> > Mostly, it is about to override the config options. If we support > > dynamic > > >> > properties, i think > > >> > it is more convenient for users without any learning curve. > > >> > `docker run flink session_jobmanager -D rest.bind-port=8081` > > >> > > >> Indeed, as mentioned before, it can be a better option. > > >> The standard entry point also decides at least what to run JM or TM. I > > >> think we will see what else makes sense to include there during the > > >> implementation. > > >> Some specifics may be more convenient to set with env vars as > Konstantin > > >> mentioned. > > >> > > >> > About the logging > > >> > Updating the `log4j-console.properties` to support multiple appender > > is a > > >> > better option. > > >> > Currently, the native K8s is suggesting users to debug the logs in > > this > > >> > way[1]. However, > > >> > there is also some problems. The stderr and stdout of JM/TM > processes > > could > > >> > not be > > >> > forwarded to the docker container console. > > >> > > >> Strange, we should check maybe there is a docker option to query the > > >> container's stderr output as well. > > >> If we forward Flink process stdout as usual in bash console, it should > > not > > >> be a problem. Why can it not be forwarded? > > >> > > >> @Konstantin Knauf > > >> > > >> For the entrypoint, have you considered to also allow setting > > configuration > > >> > via environment variables as in "docker run -e > > FLINK_REST_BIN_PORT=8081 > > >> > ..."? This is quite common and more flexible, e.g. it makes it very > > easy to > > >> > pass values of Kubernetes Secrets into the Flink configuration. > > >> > > >> This is indeed an interesting option to pass arguments to the entry > > point > > >> in general. > > >> For the config options, the dynamic args can be a better option as > > >> mentioned above. > > >> > > >> With respect to logging, I would opt to keep this very basic and to > only > > >> > support logging to the console (maybe with a fix for the web user > > >> > interface). For everything else, users can easily build their own > > images > > >> > based on library/flink (provide the dependencies, change the logging > > >> > configuration). > > >> > > >> agree > > >> > > >> Thanks, > > >> Andrey > > >> > > >> On Sun, Mar 8, 2020 at 8:55 PM Konstantin Knauf < > > [hidden email]> > > >> wrote: > > >> > > >> > Hi Andrey, > > >> > > > >> > thanks a lot for this proposal. The variety of Docker files in the > > project > > >> > has been causing quite some confusion. > > >> > > > >> > For the entrypoint, have you considered to also allow setting > > >> > configuration via environment variables as in "docker run -e > > >> > FLINK_REST_BIN_PORT=8081 ..."? This is quite common and more > > flexible, e.g. > > >> > it makes it very easy to pass values of Kubernetes Secrets into the > > Flink > > >> > configuration. > > >> > > > >> > With respect to logging, I would opt to keep this very basic and to > > only > > >> > support logging to the console (maybe with a fix for the web user > > >> > interface). For everything else, users can easily build their own > > images > > >> > based on library/flink (provide the dependencies, change the logging > > >> > configuration). > > >> > > > >> > Cheers, > > >> > > > >> > Konstantin > > >> > > > >> > > > >> > On Thu, Mar 5, 2020 at 11:01 AM Yang Wang <[hidden email]> > > wrote: > > >> > > > >> >> Hi Andrey, > > >> >> > > >> >> > > >> >> Thanks for driving this significant FLIP. From the user ML, we > could > > also > > >> >> know there are > > >> >> many users running Flink in container environment. Then the docker > > image > > >> >> will be the > > >> >> very basic requirement. Just as you say, we should provide a > unified > > >> >> place for all various > > >> >> usage(e.g. session, job, native k8s, swarm, etc.). > > >> >> > > >> >> > > >> >> > About docker utils > > >> >> > > >> >> I really like the idea to provide some utils for the docker file > and > > >> >> entry point. The > > >> >> `flink_docker_utils` will help to build the image easier. I am not > > sure > > >> >> about the > > >> >> `flink_docker_utils start_jobmaster`. Do you mean when we build a > > docker > > >> >> image, we > > >> >> need to add `RUN flink_docker_utils start_jobmaster` in the docker > > file? > > >> >> Why do we need this? > > >> >> > > >> >> > > >> >> > About docker entry point > > >> >> > > >> >> I agree with you that the docker entry point could more powerful > with > > >> >> more functionality. > > >> >> Mostly, it is about to override the config options. If we support > > dynamic > > >> >> properties, i think > > >> >> it is more convenient for users without any learning curve. > > >> >> `docker run flink session_jobmanager -D rest.bind-port=8081` > > >> >> > > >> >> > > >> >> > About the logging > > >> >> > > >> >> Updating the `log4j-console.properties` to support multiple > appender > > is a > > >> >> better option. > > >> >> Currently, the native K8s is suggesting users to debug the logs in > > this > > >> >> way[1]. However, > > >> >> there is also some problems. The stderr and stdout of JM/TM > processes > > >> >> could not be > > >> >> forwarded to the docker container console. > > >> >> > > >> >> > > >> >> [1]. > > >> >> > > > https://ci.apache.org/projects/flink/flink-docs-master/ops/deployment/native_kubernetes.html#log-files > > >> >> > > >> >> > > >> >> Best, > > >> >> Yang > > >> >> > > >> >> > > >> >> > > >> >> > > >> >> Andrey Zagrebin <[hidden email]> 于2020年3月4日周三 下午5:34写道: > > >> >> > > >> >>> Hi All, > > >> >>> > > >> >>> If you have ever touched the docker topic in Flink, you > > >> >>> probably noticed that we have multiple places in docs and repos > > which > > >> >>> address its various concerns. > > >> >>> > > >> >>> We have prepared a FLIP [1] to simplify the perception of docker > > topic in > > >> >>> Flink by users. It mostly advocates for an approach of extending > > official > > >> >>> Flink image from the docker hub. For convenience, it can come with > > a set > > >> >>> of > > >> >>> bash utilities and documented examples of their usage. The > utilities > > >> >>> allow > > >> >>> to: > > >> >>> > > >> >>> - run the docker image in various modes (single job, session > > master, > > >> >>> task manager etc) > > >> >>> - customise the extending Dockerfile > > >> >>> - and its entry point > > >> >>> > > >> >>> Eventually, the FLIP suggests to remove all other user facing > > Dockerfiles > > >> >>> and building scripts from Flink repo, move all docker docs to > > >> >>> apache/flink-docker and adjust existing docker use cases to refer > > to this > > >> >>> new approach (mostly Kubernetes now). > > >> >>> > > >> >>> The first contributed version of Flink docker integration also > > contained > > >> >>> example and docs for the integration with Bluemix in IBM cloud. We > > also > > >> >>> suggest to maintain it outside of Flink repository (cc Markus > > Müller). > > >> >>> > > >> >>> Thanks, > > >> >>> Andrey > > >> >>> > > >> >>> [1] > > >> >>> > > >> >>> > > > https://cwiki.apache.org/confluence/display/FLINK/FLIP-111%3A+Docker+image+unification > > >> >>> > > >> >> > > >> > > > >> > -- > > >> > > > >> > Konstantin Knauf | Head of Product > > >> > > > >> > +49 160 91394525 > > >> > > > >> > > > >> > Follow us @VervericaData Ververica <https://www.ververica.com/> > > >> > > > >> > > > >> > -- > > >> > > > >> > Join Flink Forward <https://flink-forward.org/> - The Apache Flink > > >> > Conference > > >> > > > >> > Stream Processing | Event Driven | Real Time > > >> > > > >> > -- > > >> > > > >> > Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany > > >> > > > >> > -- > > >> > Ververica GmbH > > >> > Registered at Amtsgericht Charlottenburg: HRB 158244 B > > >> > Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, > > Ji > > >> > (Tony) Cheng > > >> > > > > |
I second Thomas that we can support both Java 8 and 11.
Best, Yangze Guo On Wed, Mar 18, 2020 at 12:12 PM Thomas Weise <[hidden email]> wrote: > > --> > > On Mon, Mar 16, 2020 at 1:58 AM Andrey Zagrebin <[hidden email]> wrote: >> >> Thanks for the further feedback Thomas and Yangze. >> >> > A generic, dynamic configuration mechanism based on environment variables >> is essential and it is already supported via envsubst and an environment >> variable that can supply a configuration fragment >> >> True, we already have this. As I understand this was introduced for >> flexibility to template a custom flink-conf.yaml with env vars, put it into >> the FLINK_PROPERTIES and merge it with the default one. >> Could we achieve the same with the dynamic properties (-Drpc.port=1234), >> passed as image args to run it, instead of FLINK_PROPERTIES? >> They could be also parametrised with env vars. This would require >> jobmanager.sh to properly propagate them to >> the StandaloneSessionClusterEntrypoint though: >> https://github.com/docker-flink/docker-flink/pull/82#issuecomment-525285552 >> cc @Till >> This would provide a unified configuration approach. >> > > How would that look like for the various use cases? The k8s operator would need to generate the -Dabc .. -Dxyz entry point command instead of setting the FLINK_PROPERTIES environment variable? Potentially that introduces additional complexity for little gain. Do most deployment platforms that support Docker containers handle the command line route well? Backward compatibility may also be a concern. > >> >> > On the flip side, attempting to support a fixed subset of configuration >> options is brittle and will probably lead to compatibility issues down the >> road >> >> I agree with it. The idea was to have just some shortcut scripted functions >> to set options in flink-conf.yaml for a custom Dockerfile or entry point >> script. >> TASK_MANAGER_NUMBER_OF_TASK_SLOTS could be set as a dynamic property of >> started JM. >> I am not sure how many users depend on it. Maybe we could remove it. >> It also looks we already have somewhat unclean state in >> the docker-entrypoint.sh where some ports are set the hardcoded values >> and then FLINK_PROPERTIES are applied potentially duplicating options in >> the result flink-conf.yaml. > > > That is indeed possible and duplicate entries from FLINK_PROPERTIES prevail. Unfortunately, the special cases you mention were already established and the generic mechanism was added later for the k8s operators. > >> >> >> I can see some potential usage of env vars as standard entry point args but >> for purposes related to something which cannot be achieved by passing entry >> point args, like changing flink-conf.yaml options. Nothing comes into my >> mind at the moment. It could be some setting specific to the running mode >> of the entry point. The mode itself can stay the first arg of the entry >> point. >> >> > I would second that it is desirable to support Java 11 >> >> > Regarding supporting JAVA 11: >> > - Not sure if it is necessary to ship JAVA. Maybe we could just change >> > the base image from openjdk:8-jre to openjdk:11-jre in template docker >> > file[1]. Correct me if I understand incorrectly. Also, I agree to move >> > this out of the scope of this FLIP if it indeed takes much extra >> > effort. >> >> This is what I meant by bumping up the Java version in the docker hub Flink >> image: >> FROM openjdk:8-jre -> FROM openjdk:11-jre >> This can be polled dependently in user mailing list. > > > That sounds reasonable as long as we can still support both Java versions (i.e. provide separate images for 8 and 11). > >> >> >> > and in general use a base image that allows the (straightforward) use of >> more recent versions of other software (Python etc.) >> >> This can be polled whether to always include some version of python into >> the docker hub image. >> A potential problem here is once it is there, it is some hassle to >> remove/change it in a custom extended Dockerfile. >> >> It would be also nice to avoid maintaining images for various combinations >> of installed Java/Scala/Python in docker hub. >> >> > Regarding building from local dist: >> > - Yes, I bring this up mostly for development purpose. Since k8s is >> > popular, I believe more and more developers would like to test their >> > work on k8s cluster. I'm not sure should all developers write a custom >> > docker file themselves in this scenario. Thus, I still prefer to >> > provide a script for devs. >> > - I agree to keep the scope of this FLIP mostly for those normal >> > users. But as far as I can see, supporting building from local dist >> > would not take much extra effort. >> > - The maven docker plugin sounds good. I'll take a look at it. >> >> I would see any scripts introduced in this FLIP also as potential building >> blocks for a custom dev Dockerfile. >> Maybe, this will be all what we need for dev images or we write a dev >> Dockerfile, highly parametrised for building a dev image. >> If scripts stay in apache/flink-docker, it is also somewhat inconvenient to >> use them in the main Flink repo but possible. >> If we move them to apache/flink then we will have to e.g. include them into >> the release to make them easily available in apache/flink-docker and >> maintain them in main repo, although they are only docker specific. >> All in all, I would say, once we implement them, we can revisit this topic. >> >> Best, >> Andrey >> >> On Wed, Mar 11, 2020 at 8:58 AM Yangze Guo <[hidden email]> wrote: >> >> > Thanks for the reply, Andrey. >> > >> > Regarding building from local dist: >> > - Yes, I bring this up mostly for development purpose. Since k8s is >> > popular, I believe more and more developers would like to test their >> > work on k8s cluster. I'm not sure should all developers write a custom >> > docker file themselves in this scenario. Thus, I still prefer to >> > provide a script for devs. >> > - I agree to keep the scope of this FLIP mostly for those normal >> > users. But as far as I can see, supporting building from local dist >> > would not take much extra effort. >> > - The maven docker plugin sounds good. I'll take a look at it. >> > >> > Regarding supporting JAVA 11: >> > - Not sure if it is necessary to ship JAVA. Maybe we could just change >> > the base image from openjdk:8-jre to openjdk:11-jre in template docker >> > file[1]. Correct me if I understand incorrectly. Also, I agree to move >> > this out of the scope of this FLIP if it indeed takes much extra >> > effort. >> > >> > Regarding the custom configuration, the mechanism that Thomas mentioned >> > LGTM. >> > >> > [1] >> > https://github.com/apache/flink-docker/blob/master/Dockerfile-debian.template >> > >> > Best, >> > Yangze Guo >> > >> > On Wed, Mar 11, 2020 at 5:52 AM Thomas Weise <[hidden email]> wrote: >> > > >> > > Thanks for working on improvements to the Flink Docker container images. >> > This will be important as more and more users are looking to adopt >> > Kubernetes and other deployment tooling that relies on Docker images. >> > > >> > > A generic, dynamic configuration mechanism based on environment >> > variables is essential and it is already supported via envsubst and an >> > environment variable that can supply a configuration fragment: >> > > >> > > >> > https://github.com/apache/flink-docker/blob/09adf2dcd99abfb6180e1e2b5b917b288e0c01f6/docker-entrypoint.sh#L88 >> > > >> > https://github.com/apache/flink-docker/blob/09adf2dcd99abfb6180e1e2b5b917b288e0c01f6/docker-entrypoint.sh#L85 >> > > >> > > This gives the necessary control for infrastructure use cases that aim >> > to supply deployment tooling other users. An example in this category this >> > is the FlinkK8sOperator: >> > > >> > > https://github.com/lyft/flinkk8soperator/tree/master/examples/wordcount >> > > >> > > On the flip side, attempting to support a fixed subset of configuration >> > options is brittle and will probably lead to compatibility issues down the >> > road: >> > > >> > > >> > https://github.com/apache/flink-docker/blob/09adf2dcd99abfb6180e1e2b5b917b288e0c01f6/docker-entrypoint.sh#L97 >> > > >> > > Besides the configuration, it may be worthwhile to see in which other >> > ways the base Docker images can provide more flexibility to incentivize >> > wider adoption. >> > > >> > > I would second that it is desirable to support Java 11 and in general >> > use a base image that allows the (straightforward) use of more recent >> > versions of other software (Python etc.) >> > > >> > > >> > https://github.com/apache/flink-docker/blob/d3416e720377e9b4c07a2d0f4591965264ac74c5/Dockerfile-debian.template#L19 >> > > >> > > Thanks, >> > > Thomas >> > > >> > > On Tue, Mar 10, 2020 at 12:26 PM Andrey Zagrebin <[hidden email]> >> > wrote: >> > >> >> > >> Hi All, >> > >> >> > >> Thanks a lot for the feedback! >> > >> >> > >> *@Yangze Guo* >> > >> >> > >> - Regarding the flink_docker_utils#install_flink function, I think it >> > >> > should also support build from local dist and build from a >> > >> > user-defined archive. >> > >> >> > >> I suppose you bring this up mostly for development purpose or powerful >> > >> users. >> > >> Most of normal users are usually interested in mainstream released >> > versions >> > >> of Flink. >> > >> Although, you are bring a valid concern, my idea was to keep scope of >> > this >> > >> FLIP mostly for those normal users. >> > >> The powerful users are usually capable to design a completely >> > >> custom Dockerfile themselves. >> > >> At the moment, we already have custom Dockerfiles e.g. for tests in >> > >> >> > flink-end-to-end-tests/test-scripts/docker-hadoop-secure-cluster/Dockerfile. >> > >> We can add something similar for development purposes and maybe >> > introduce a >> > >> special maven goal. There is a maven docker plugin, afaik. >> > >> I will add this to FLIP as next step. >> > >> >> > >> - It seems that the install_shaded_hadoop could be an option of >> > >> > install_flink >> > >> >> > >> I woud rather think about this as a separate independent optional step. >> > >> >> > >> - Should we support JAVA 11? Currently, most of the docker file based on >> > >> > JAVA 8. >> > >> >> > >> Indeed, it is a valid concern. Java version is a fundamental property of >> > >> the docker image. >> > >> To customise this in the current mainstream image is difficult, this >> > would >> > >> require to ship it w/o Java at all. >> > >> Or this is a separate discussion whether we want to distribute docker >> > hub >> > >> images with different Java versions or just bump it to Java 11. >> > >> This should be easy in a custom Dockerfile for development purposes >> > though >> > >> as mentioned before. >> > >> >> > >> - I do not understand how to set config options through >> > >> >> > >> "flink_docker_utils configure"? Does this step happen during the image >> > >> > build or the container start? If it happens during the image build, >> > >> > there would be a new image every time we change the config. If it just >> > >> > a part of the container entrypoint, I think there is no need to add a >> > >> > configure command, we could just add all dynamic config options to the >> > >> > args list of "start_jobmaster"/"start_session_jobmanager". Am I >> > >> > understanding this correctly? >> > >> >> > >> `flink_docker_utils configure ...` can be called everywhere: >> > >> - while building a custom image (`RUN flink_docker_utils configure ..`) >> > by >> > >> extending our base image from docker hub (`from flink`) >> > >> - in a custom entry point as well >> > >> I will check this but if user can also pass a dynamic config option it >> > also >> > >> sounds like a good option >> > >> Our standard entry point script in base image could just properly >> > forward >> > >> the arguments to the Flink process. >> > >> >> > >> @Yang Wang >> > >> >> > >> > About docker utils >> > >> > I really like the idea to provide some utils for the docker file and >> > entry >> > >> > point. The >> > >> > `flink_docker_utils` will help to build the image easier. I am not >> > sure >> > >> > about the >> > >> > `flink_docker_utils start_jobmaster`. Do you mean when we build a >> > docker >> > >> > image, we >> > >> > need to add `RUN flink_docker_utils start_jobmaster` in the docker >> > file? >> > >> > Why do we need this? >> > >> >> > >> This is a scripted action to start JM. It can be called everywhere. >> > >> Indeed, it does not make too much sense to run it in Dockerfile. >> > >> Mostly, the idea was to use in a custom entry point. When our base >> > docker >> > >> hub image is started its entry point can be also completely overridden. >> > >> The actions are also sorted in the FLIP: for Dockerfile or for entry >> > point. >> > >> E.g. our standard entry point script in the base docker hub image can >> > >> already use it. >> > >> Anyways, it was just an example, the details are to be defined in Jira, >> > imo. >> > >> >> > >> > About docker entry point >> > >> > I agree with you that the docker entry point could more powerful with >> > more >> > >> > functionality. >> > >> > Mostly, it is about to override the config options. If we support >> > dynamic >> > >> > properties, i think >> > >> > it is more convenient for users without any learning curve. >> > >> > `docker run flink session_jobmanager -D rest.bind-port=8081` >> > >> >> > >> Indeed, as mentioned before, it can be a better option. >> > >> The standard entry point also decides at least what to run JM or TM. I >> > >> think we will see what else makes sense to include there during the >> > >> implementation. >> > >> Some specifics may be more convenient to set with env vars as Konstantin >> > >> mentioned. >> > >> >> > >> > About the logging >> > >> > Updating the `log4j-console.properties` to support multiple appender >> > is a >> > >> > better option. >> > >> > Currently, the native K8s is suggesting users to debug the logs in >> > this >> > >> > way[1]. However, >> > >> > there is also some problems. The stderr and stdout of JM/TM processes >> > could >> > >> > not be >> > >> > forwarded to the docker container console. >> > >> >> > >> Strange, we should check maybe there is a docker option to query the >> > >> container's stderr output as well. >> > >> If we forward Flink process stdout as usual in bash console, it should >> > not >> > >> be a problem. Why can it not be forwarded? >> > >> >> > >> @Konstantin Knauf >> > >> >> > >> For the entrypoint, have you considered to also allow setting >> > configuration >> > >> > via environment variables as in "docker run -e >> > FLINK_REST_BIN_PORT=8081 >> > >> > ..."? This is quite common and more flexible, e.g. it makes it very >> > easy to >> > >> > pass values of Kubernetes Secrets into the Flink configuration. >> > >> >> > >> This is indeed an interesting option to pass arguments to the entry >> > point >> > >> in general. >> > >> For the config options, the dynamic args can be a better option as >> > >> mentioned above. >> > >> >> > >> With respect to logging, I would opt to keep this very basic and to only >> > >> > support logging to the console (maybe with a fix for the web user >> > >> > interface). For everything else, users can easily build their own >> > images >> > >> > based on library/flink (provide the dependencies, change the logging >> > >> > configuration). >> > >> >> > >> agree >> > >> >> > >> Thanks, >> > >> Andrey >> > >> >> > >> On Sun, Mar 8, 2020 at 8:55 PM Konstantin Knauf < >> > [hidden email]> >> > >> wrote: >> > >> >> > >> > Hi Andrey, >> > >> > >> > >> > thanks a lot for this proposal. The variety of Docker files in the >> > project >> > >> > has been causing quite some confusion. >> > >> > >> > >> > For the entrypoint, have you considered to also allow setting >> > >> > configuration via environment variables as in "docker run -e >> > >> > FLINK_REST_BIN_PORT=8081 ..."? This is quite common and more >> > flexible, e.g. >> > >> > it makes it very easy to pass values of Kubernetes Secrets into the >> > Flink >> > >> > configuration. >> > >> > >> > >> > With respect to logging, I would opt to keep this very basic and to >> > only >> > >> > support logging to the console (maybe with a fix for the web user >> > >> > interface). For everything else, users can easily build their own >> > images >> > >> > based on library/flink (provide the dependencies, change the logging >> > >> > configuration). >> > >> > >> > >> > Cheers, >> > >> > >> > >> > Konstantin >> > >> > >> > >> > >> > >> > On Thu, Mar 5, 2020 at 11:01 AM Yang Wang <[hidden email]> >> > wrote: >> > >> > >> > >> >> Hi Andrey, >> > >> >> >> > >> >> >> > >> >> Thanks for driving this significant FLIP. From the user ML, we could >> > also >> > >> >> know there are >> > >> >> many users running Flink in container environment. Then the docker >> > image >> > >> >> will be the >> > >> >> very basic requirement. Just as you say, we should provide a unified >> > >> >> place for all various >> > >> >> usage(e.g. session, job, native k8s, swarm, etc.). >> > >> >> >> > >> >> >> > >> >> > About docker utils >> > >> >> >> > >> >> I really like the idea to provide some utils for the docker file and >> > >> >> entry point. The >> > >> >> `flink_docker_utils` will help to build the image easier. I am not >> > sure >> > >> >> about the >> > >> >> `flink_docker_utils start_jobmaster`. Do you mean when we build a >> > docker >> > >> >> image, we >> > >> >> need to add `RUN flink_docker_utils start_jobmaster` in the docker >> > file? >> > >> >> Why do we need this? >> > >> >> >> > >> >> >> > >> >> > About docker entry point >> > >> >> >> > >> >> I agree with you that the docker entry point could more powerful with >> > >> >> more functionality. >> > >> >> Mostly, it is about to override the config options. If we support >> > dynamic >> > >> >> properties, i think >> > >> >> it is more convenient for users without any learning curve. >> > >> >> `docker run flink session_jobmanager -D rest.bind-port=8081` >> > >> >> >> > >> >> >> > >> >> > About the logging >> > >> >> >> > >> >> Updating the `log4j-console.properties` to support multiple appender >> > is a >> > >> >> better option. >> > >> >> Currently, the native K8s is suggesting users to debug the logs in >> > this >> > >> >> way[1]. However, >> > >> >> there is also some problems. The stderr and stdout of JM/TM processes >> > >> >> could not be >> > >> >> forwarded to the docker container console. >> > >> >> >> > >> >> >> > >> >> [1]. >> > >> >> >> > https://ci.apache.org/projects/flink/flink-docs-master/ops/deployment/native_kubernetes.html#log-files >> > >> >> >> > >> >> >> > >> >> Best, >> > >> >> Yang >> > >> >> >> > >> >> >> > >> >> >> > >> >> >> > >> >> Andrey Zagrebin <[hidden email]> 于2020年3月4日周三 下午5:34写道: >> > >> >> >> > >> >>> Hi All, >> > >> >>> >> > >> >>> If you have ever touched the docker topic in Flink, you >> > >> >>> probably noticed that we have multiple places in docs and repos >> > which >> > >> >>> address its various concerns. >> > >> >>> >> > >> >>> We have prepared a FLIP [1] to simplify the perception of docker >> > topic in >> > >> >>> Flink by users. It mostly advocates for an approach of extending >> > official >> > >> >>> Flink image from the docker hub. For convenience, it can come with >> > a set >> > >> >>> of >> > >> >>> bash utilities and documented examples of their usage. The utilities >> > >> >>> allow >> > >> >>> to: >> > >> >>> >> > >> >>> - run the docker image in various modes (single job, session >> > master, >> > >> >>> task manager etc) >> > >> >>> - customise the extending Dockerfile >> > >> >>> - and its entry point >> > >> >>> >> > >> >>> Eventually, the FLIP suggests to remove all other user facing >> > Dockerfiles >> > >> >>> and building scripts from Flink repo, move all docker docs to >> > >> >>> apache/flink-docker and adjust existing docker use cases to refer >> > to this >> > >> >>> new approach (mostly Kubernetes now). >> > >> >>> >> > >> >>> The first contributed version of Flink docker integration also >> > contained >> > >> >>> example and docs for the integration with Bluemix in IBM cloud. We >> > also >> > >> >>> suggest to maintain it outside of Flink repository (cc Markus >> > Müller). >> > >> >>> >> > >> >>> Thanks, >> > >> >>> Andrey >> > >> >>> >> > >> >>> [1] >> > >> >>> >> > >> >>> >> > https://cwiki.apache.org/confluence/display/FLINK/FLIP-111%3A+Docker+image+unification >> > >> >>> >> > >> >> >> > >> > >> > >> > -- >> > >> > >> > >> > Konstantin Knauf | Head of Product >> > >> > >> > >> > +49 160 91394525 >> > >> > >> > >> > >> > >> > Follow us @VervericaData Ververica <https://www.ververica.com/> >> > >> > >> > >> > >> > >> > -- >> > >> > >> > >> > Join Flink Forward <https://flink-forward.org/> - The Apache Flink >> > >> > Conference >> > >> > >> > >> > Stream Processing | Event Driven | Real Time >> > >> > >> > >> > -- >> > >> > >> > >> > Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany >> > >> > >> > >> > -- >> > >> > Ververica GmbH >> > >> > Registered at Amtsgericht Charlottenburg: HRB 158244 B >> > >> > Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, >> > Ji >> > >> > (Tony) Cheng >> > >> > >> > |
Thanks for creating this FLIP Andrey. I like the general idea pretty much.
I tried to group some of the above-mentioned points in order to give my 50 cent. # Configuration How to configure the Flink process seems to be the biggest question. Due to historical reasons we have a plethora of different ways on how to configure the Flink Docker image. I think the most general approach would be the configuration via environment variables where one can specify env variables with the form FLINK_<CONFIG_KEY>=<VALUE> which will overwrite <CONFIG_KEY> with <VALUE> in the effective configuration. However, this is something which deserves a separate effort and is out of scope for this proposal. The next best thing for configuring Flink (not the Flink process) would be dynamic program arguments. For this to work, we would need to extend the jobmanager.sh and taskmanager.sh scripts. I think this is strictly speaking also an orthogonal task and could happen as a follow up/independently. This leaves us with the envsubst and FLINK_PROPERTIES approaches. Even though I'm not a huge fan of these approaches, I think we should still support them for backwards compatibility reasons. Once we support configuration via env variables we should deprecate these methods and remove them in a subsequent release. Given this, I am bit unsure about introducing yet another way via flink_docker_utils configure "option.name" “value”. I think we should only offer this option if we are sure that we want to keep it in the future and that it won't be superceded by the env variables approach. Otherwise it will only add more maintenance burden. Long story short, with the existing configuration options (envsubts, FLINK_PROPERTIES) we can already configure the Flink process and Flink itself. Since maintaining backwards compatibility is important, we could rely on these mechanisms until we have proper env variable configuration and don't have to introduce a new way to change the configuration. # Logging & Stdout/err ## Logging I think Konstantin is right and we should provide a log4j.properties file which, per default, specifies the file and console appender. We could add a special log4j.properties file to apache/flink-docker which we include in the Dockerfile. This approach will give users the most flexibility w/o relying on magic (e.g. tailing the log files after starting the process in the background). ## Stdout/err I think for printing the stdout/err output to STDOUT/ERR and to capture it in a file there are solutions. For example, one could use `program &2>1 | tee flink-user-taskexecutor.out` to achieve this. # Java version I agree that it would be nice to also offer a Java 11 Dockerfile. For the sake of limiting the scope of this proposal I would suggest to do this as a follow up issue. # Dev version Tooling to create a Docker image from the current Flink repository is indeed very nice for development. As Andrey suggested, I think this would be a good follow up for this proposal. I don't think that Andrey's current proposal would block any future developments in this direction. # Scripts At the moment, I would be in favour of placing the Dockerfile scripts under apache/flink-docker since they belong more to the Dockerfile than to Flink's binary distribution. If we see that we might be able to reuse them for the developer Dockerfile, then we can still move them to the Flink repository. I would refrain from offering special commands to set individual configuration options (e.g., flink_docker_utils set_web_ui_port 8081). It should be fine enough to do it via flink_docker-utils conifgure rest.port 8081 if we cannot solve it via the general configuration mechanism. Cheers, Till On Wed, Mar 18, 2020 at 6:38 AM Yangze Guo <[hidden email]> wrote: > I second Thomas that we can support both Java 8 and 11. > > Best, > Yangze Guo > > On Wed, Mar 18, 2020 at 12:12 PM Thomas Weise <[hidden email]> wrote: > > > > --> > > > > On Mon, Mar 16, 2020 at 1:58 AM Andrey Zagrebin <[hidden email]> > wrote: > >> > >> Thanks for the further feedback Thomas and Yangze. > >> > >> > A generic, dynamic configuration mechanism based on environment > variables > >> is essential and it is already supported via envsubst and an environment > >> variable that can supply a configuration fragment > >> > >> True, we already have this. As I understand this was introduced for > >> flexibility to template a custom flink-conf.yaml with env vars, put it > into > >> the FLINK_PROPERTIES and merge it with the default one. > >> Could we achieve the same with the dynamic properties (-Drpc.port=1234), > >> passed as image args to run it, instead of FLINK_PROPERTIES? > >> They could be also parametrised with env vars. This would require > >> jobmanager.sh to properly propagate them to > >> the StandaloneSessionClusterEntrypoint though: > >> > https://github.com/docker-flink/docker-flink/pull/82#issuecomment-525285552 > >> cc @Till > >> This would provide a unified configuration approach. > >> > > > > How would that look like for the various use cases? The k8s operator > would need to generate the -Dabc .. -Dxyz entry point command instead of > setting the FLINK_PROPERTIES environment variable? Potentially that > introduces additional complexity for little gain. Do most deployment > platforms that support Docker containers handle the command line route > well? Backward compatibility may also be a concern. > > > >> > >> > On the flip side, attempting to support a fixed subset of > configuration > >> options is brittle and will probably lead to compatibility issues down > the > >> road > >> > >> I agree with it. The idea was to have just some shortcut scripted > functions > >> to set options in flink-conf.yaml for a custom Dockerfile or entry point > >> script. > >> TASK_MANAGER_NUMBER_OF_TASK_SLOTS could be set as a dynamic property of > >> started JM. > >> I am not sure how many users depend on it. Maybe we could remove it. > >> It also looks we already have somewhat unclean state in > >> the docker-entrypoint.sh where some ports are set the hardcoded values > >> and then FLINK_PROPERTIES are applied potentially duplicating options in > >> the result flink-conf.yaml. > > > > > > That is indeed possible and duplicate entries from FLINK_PROPERTIES > prevail. Unfortunately, the special cases you mention were already > established and the generic mechanism was added later for the k8s operators. > > > >> > >> > >> I can see some potential usage of env vars as standard entry point args > but > >> for purposes related to something which cannot be achieved by passing > entry > >> point args, like changing flink-conf.yaml options. Nothing comes into my > >> mind at the moment. It could be some setting specific to the running > mode > >> of the entry point. The mode itself can stay the first arg of the entry > >> point. > >> > >> > I would second that it is desirable to support Java 11 > >> > >> > Regarding supporting JAVA 11: > >> > - Not sure if it is necessary to ship JAVA. Maybe we could just change > >> > the base image from openjdk:8-jre to openjdk:11-jre in template docker > >> > file[1]. Correct me if I understand incorrectly. Also, I agree to move > >> > this out of the scope of this FLIP if it indeed takes much extra > >> > effort. > >> > >> This is what I meant by bumping up the Java version in the docker hub > Flink > >> image: > >> FROM openjdk:8-jre -> FROM openjdk:11-jre > >> This can be polled dependently in user mailing list. > > > > > > That sounds reasonable as long as we can still support both Java > versions (i.e. provide separate images for 8 and 11). > > > >> > >> > >> > and in general use a base image that allows the (straightforward) use > of > >> more recent versions of other software (Python etc.) > >> > >> This can be polled whether to always include some version of python into > >> the docker hub image. > >> A potential problem here is once it is there, it is some hassle to > >> remove/change it in a custom extended Dockerfile. > >> > >> It would be also nice to avoid maintaining images for various > combinations > >> of installed Java/Scala/Python in docker hub. > >> > >> > Regarding building from local dist: > >> > - Yes, I bring this up mostly for development purpose. Since k8s is > >> > popular, I believe more and more developers would like to test their > >> > work on k8s cluster. I'm not sure should all developers write a custom > >> > docker file themselves in this scenario. Thus, I still prefer to > >> > provide a script for devs. > >> > - I agree to keep the scope of this FLIP mostly for those normal > >> > users. But as far as I can see, supporting building from local dist > >> > would not take much extra effort. > >> > - The maven docker plugin sounds good. I'll take a look at it. > >> > >> I would see any scripts introduced in this FLIP also as potential > building > >> blocks for a custom dev Dockerfile. > >> Maybe, this will be all what we need for dev images or we write a dev > >> Dockerfile, highly parametrised for building a dev image. > >> If scripts stay in apache/flink-docker, it is also somewhat > inconvenient to > >> use them in the main Flink repo but possible. > >> If we move them to apache/flink then we will have to e.g. include them > into > >> the release to make them easily available in apache/flink-docker and > >> maintain them in main repo, although they are only docker specific. > >> All in all, I would say, once we implement them, we can revisit this > topic. > >> > >> Best, > >> Andrey > >> > >> On Wed, Mar 11, 2020 at 8:58 AM Yangze Guo <[hidden email]> wrote: > >> > >> > Thanks for the reply, Andrey. > >> > > >> > Regarding building from local dist: > >> > - Yes, I bring this up mostly for development purpose. Since k8s is > >> > popular, I believe more and more developers would like to test their > >> > work on k8s cluster. I'm not sure should all developers write a custom > >> > docker file themselves in this scenario. Thus, I still prefer to > >> > provide a script for devs. > >> > - I agree to keep the scope of this FLIP mostly for those normal > >> > users. But as far as I can see, supporting building from local dist > >> > would not take much extra effort. > >> > - The maven docker plugin sounds good. I'll take a look at it. > >> > > >> > Regarding supporting JAVA 11: > >> > - Not sure if it is necessary to ship JAVA. Maybe we could just change > >> > the base image from openjdk:8-jre to openjdk:11-jre in template docker > >> > file[1]. Correct me if I understand incorrectly. Also, I agree to move > >> > this out of the scope of this FLIP if it indeed takes much extra > >> > effort. > >> > > >> > Regarding the custom configuration, the mechanism that Thomas > mentioned > >> > LGTM. > >> > > >> > [1] > >> > > https://github.com/apache/flink-docker/blob/master/Dockerfile-debian.template > >> > > >> > Best, > >> > Yangze Guo > >> > > >> > On Wed, Mar 11, 2020 at 5:52 AM Thomas Weise <[hidden email]> wrote: > >> > > > >> > > Thanks for working on improvements to the Flink Docker container > images. > >> > This will be important as more and more users are looking to adopt > >> > Kubernetes and other deployment tooling that relies on Docker images. > >> > > > >> > > A generic, dynamic configuration mechanism based on environment > >> > variables is essential and it is already supported via envsubst and an > >> > environment variable that can supply a configuration fragment: > >> > > > >> > > > >> > > https://github.com/apache/flink-docker/blob/09adf2dcd99abfb6180e1e2b5b917b288e0c01f6/docker-entrypoint.sh#L88 > >> > > > >> > > https://github.com/apache/flink-docker/blob/09adf2dcd99abfb6180e1e2b5b917b288e0c01f6/docker-entrypoint.sh#L85 > >> > > > >> > > This gives the necessary control for infrastructure use cases that > aim > >> > to supply deployment tooling other users. An example in this category > this > >> > is the FlinkK8sOperator: > >> > > > >> > > > https://github.com/lyft/flinkk8soperator/tree/master/examples/wordcount > >> > > > >> > > On the flip side, attempting to support a fixed subset of > configuration > >> > options is brittle and will probably lead to compatibility issues > down the > >> > road: > >> > > > >> > > > >> > > https://github.com/apache/flink-docker/blob/09adf2dcd99abfb6180e1e2b5b917b288e0c01f6/docker-entrypoint.sh#L97 > >> > > > >> > > Besides the configuration, it may be worthwhile to see in which > other > >> > ways the base Docker images can provide more flexibility to > incentivize > >> > wider adoption. > >> > > > >> > > I would second that it is desirable to support Java 11 and in > general > >> > use a base image that allows the (straightforward) use of more recent > >> > versions of other software (Python etc.) > >> > > > >> > > > >> > > https://github.com/apache/flink-docker/blob/d3416e720377e9b4c07a2d0f4591965264ac74c5/Dockerfile-debian.template#L19 > >> > > > >> > > Thanks, > >> > > Thomas > >> > > > >> > > On Tue, Mar 10, 2020 at 12:26 PM Andrey Zagrebin < > [hidden email]> > >> > wrote: > >> > >> > >> > >> Hi All, > >> > >> > >> > >> Thanks a lot for the feedback! > >> > >> > >> > >> *@Yangze Guo* > >> > >> > >> > >> - Regarding the flink_docker_utils#install_flink function, I think > it > >> > >> > should also support build from local dist and build from a > >> > >> > user-defined archive. > >> > >> > >> > >> I suppose you bring this up mostly for development purpose or > powerful > >> > >> users. > >> > >> Most of normal users are usually interested in mainstream released > >> > versions > >> > >> of Flink. > >> > >> Although, you are bring a valid concern, my idea was to keep scope > of > >> > this > >> > >> FLIP mostly for those normal users. > >> > >> The powerful users are usually capable to design a completely > >> > >> custom Dockerfile themselves. > >> > >> At the moment, we already have custom Dockerfiles e.g. for tests in > >> > >> > >> > > flink-end-to-end-tests/test-scripts/docker-hadoop-secure-cluster/Dockerfile. > >> > >> We can add something similar for development purposes and maybe > >> > introduce a > >> > >> special maven goal. There is a maven docker plugin, afaik. > >> > >> I will add this to FLIP as next step. > >> > >> > >> > >> - It seems that the install_shaded_hadoop could be an option of > >> > >> > install_flink > >> > >> > >> > >> I woud rather think about this as a separate independent optional > step. > >> > >> > >> > >> - Should we support JAVA 11? Currently, most of the docker file > based on > >> > >> > JAVA 8. > >> > >> > >> > >> Indeed, it is a valid concern. Java version is a fundamental > property of > >> > >> the docker image. > >> > >> To customise this in the current mainstream image is difficult, > this > >> > would > >> > >> require to ship it w/o Java at all. > >> > >> Or this is a separate discussion whether we want to distribute > docker > >> > hub > >> > >> images with different Java versions or just bump it to Java 11. > >> > >> This should be easy in a custom Dockerfile for development purposes > >> > though > >> > >> as mentioned before. > >> > >> > >> > >> - I do not understand how to set config options through > >> > >> > >> > >> "flink_docker_utils configure"? Does this step happen during the > image > >> > >> > build or the container start? If it happens during the image > build, > >> > >> > there would be a new image every time we change the config. If > it just > >> > >> > a part of the container entrypoint, I think there is no need to > add a > >> > >> > configure command, we could just add all dynamic config options > to the > >> > >> > args list of "start_jobmaster"/"start_session_jobmanager". Am I > >> > >> > understanding this correctly? > >> > >> > >> > >> `flink_docker_utils configure ...` can be called everywhere: > >> > >> - while building a custom image (`RUN flink_docker_utils configure > ..`) > >> > by > >> > >> extending our base image from docker hub (`from flink`) > >> > >> - in a custom entry point as well > >> > >> I will check this but if user can also pass a dynamic config > option it > >> > also > >> > >> sounds like a good option > >> > >> Our standard entry point script in base image could just properly > >> > forward > >> > >> the arguments to the Flink process. > >> > >> > >> > >> @Yang Wang > >> > >> > >> > >> > About docker utils > >> > >> > I really like the idea to provide some utils for the docker file > and > >> > entry > >> > >> > point. The > >> > >> > `flink_docker_utils` will help to build the image easier. I am > not > >> > sure > >> > >> > about the > >> > >> > `flink_docker_utils start_jobmaster`. Do you mean when we build a > >> > docker > >> > >> > image, we > >> > >> > need to add `RUN flink_docker_utils start_jobmaster` in the > docker > >> > file? > >> > >> > Why do we need this? > >> > >> > >> > >> This is a scripted action to start JM. It can be called everywhere. > >> > >> Indeed, it does not make too much sense to run it in Dockerfile. > >> > >> Mostly, the idea was to use in a custom entry point. When our base > >> > docker > >> > >> hub image is started its entry point can be also completely > overridden. > >> > >> The actions are also sorted in the FLIP: for Dockerfile or for > entry > >> > point. > >> > >> E.g. our standard entry point script in the base docker hub image > can > >> > >> already use it. > >> > >> Anyways, it was just an example, the details are to be defined in > Jira, > >> > imo. > >> > >> > >> > >> > About docker entry point > >> > >> > I agree with you that the docker entry point could more powerful > with > >> > more > >> > >> > functionality. > >> > >> > Mostly, it is about to override the config options. If we support > >> > dynamic > >> > >> > properties, i think > >> > >> > it is more convenient for users without any learning curve. > >> > >> > `docker run flink session_jobmanager -D rest.bind-port=8081` > >> > >> > >> > >> Indeed, as mentioned before, it can be a better option. > >> > >> The standard entry point also decides at least what to run JM or > TM. I > >> > >> think we will see what else makes sense to include there during the > >> > >> implementation. > >> > >> Some specifics may be more convenient to set with env vars as > Konstantin > >> > >> mentioned. > >> > >> > >> > >> > About the logging > >> > >> > Updating the `log4j-console.properties` to support multiple > appender > >> > is a > >> > >> > better option. > >> > >> > Currently, the native K8s is suggesting users to debug the logs > in > >> > this > >> > >> > way[1]. However, > >> > >> > there is also some problems. The stderr and stdout of JM/TM > processes > >> > could > >> > >> > not be > >> > >> > forwarded to the docker container console. > >> > >> > >> > >> Strange, we should check maybe there is a docker option to query > the > >> > >> container's stderr output as well. > >> > >> If we forward Flink process stdout as usual in bash console, it > should > >> > not > >> > >> be a problem. Why can it not be forwarded? > >> > >> > >> > >> @Konstantin Knauf > >> > >> > >> > >> For the entrypoint, have you considered to also allow setting > >> > configuration > >> > >> > via environment variables as in "docker run -e > >> > FLINK_REST_BIN_PORT=8081 > >> > >> > ..."? This is quite common and more flexible, e.g. it makes it > very > >> > easy to > >> > >> > pass values of Kubernetes Secrets into the Flink configuration. > >> > >> > >> > >> This is indeed an interesting option to pass arguments to the entry > >> > point > >> > >> in general. > >> > >> For the config options, the dynamic args can be a better option as > >> > >> mentioned above. > >> > >> > >> > >> With respect to logging, I would opt to keep this very basic and > to only > >> > >> > support logging to the console (maybe with a fix for the web user > >> > >> > interface). For everything else, users can easily build their own > >> > images > >> > >> > based on library/flink (provide the dependencies, change the > logging > >> > >> > configuration). > >> > >> > >> > >> agree > >> > >> > >> > >> Thanks, > >> > >> Andrey > >> > >> > >> > >> On Sun, Mar 8, 2020 at 8:55 PM Konstantin Knauf < > >> > [hidden email]> > >> > >> wrote: > >> > >> > >> > >> > Hi Andrey, > >> > >> > > >> > >> > thanks a lot for this proposal. The variety of Docker files in > the > >> > project > >> > >> > has been causing quite some confusion. > >> > >> > > >> > >> > For the entrypoint, have you considered to also allow setting > >> > >> > configuration via environment variables as in "docker run -e > >> > >> > FLINK_REST_BIN_PORT=8081 ..."? This is quite common and more > >> > flexible, e.g. > >> > >> > it makes it very easy to pass values of Kubernetes Secrets into > the > >> > Flink > >> > >> > configuration. > >> > >> > > >> > >> > With respect to logging, I would opt to keep this very basic and > to > >> > only > >> > >> > support logging to the console (maybe with a fix for the web user > >> > >> > interface). For everything else, users can easily build their own > >> > images > >> > >> > based on library/flink (provide the dependencies, change the > logging > >> > >> > configuration). > >> > >> > > >> > >> > Cheers, > >> > >> > > >> > >> > Konstantin > >> > >> > > >> > >> > > >> > >> > On Thu, Mar 5, 2020 at 11:01 AM Yang Wang <[hidden email] > > > >> > wrote: > >> > >> > > >> > >> >> Hi Andrey, > >> > >> >> > >> > >> >> > >> > >> >> Thanks for driving this significant FLIP. From the user ML, we > could > >> > also > >> > >> >> know there are > >> > >> >> many users running Flink in container environment. Then the > docker > >> > image > >> > >> >> will be the > >> > >> >> very basic requirement. Just as you say, we should provide a > unified > >> > >> >> place for all various > >> > >> >> usage(e.g. session, job, native k8s, swarm, etc.). > >> > >> >> > >> > >> >> > >> > >> >> > About docker utils > >> > >> >> > >> > >> >> I really like the idea to provide some utils for the docker > file and > >> > >> >> entry point. The > >> > >> >> `flink_docker_utils` will help to build the image easier. I am > not > >> > sure > >> > >> >> about the > >> > >> >> `flink_docker_utils start_jobmaster`. Do you mean when we build > a > >> > docker > >> > >> >> image, we > >> > >> >> need to add `RUN flink_docker_utils start_jobmaster` in the > docker > >> > file? > >> > >> >> Why do we need this? > >> > >> >> > >> > >> >> > >> > >> >> > About docker entry point > >> > >> >> > >> > >> >> I agree with you that the docker entry point could more > powerful with > >> > >> >> more functionality. > >> > >> >> Mostly, it is about to override the config options. If we > support > >> > dynamic > >> > >> >> properties, i think > >> > >> >> it is more convenient for users without any learning curve. > >> > >> >> `docker run flink session_jobmanager -D rest.bind-port=8081` > >> > >> >> > >> > >> >> > >> > >> >> > About the logging > >> > >> >> > >> > >> >> Updating the `log4j-console.properties` to support multiple > appender > >> > is a > >> > >> >> better option. > >> > >> >> Currently, the native K8s is suggesting users to debug the logs > in > >> > this > >> > >> >> way[1]. However, > >> > >> >> there is also some problems. The stderr and stdout of JM/TM > processes > >> > >> >> could not be > >> > >> >> forwarded to the docker container console. > >> > >> >> > >> > >> >> > >> > >> >> [1]. > >> > >> >> > >> > > https://ci.apache.org/projects/flink/flink-docs-master/ops/deployment/native_kubernetes.html#log-files > >> > >> >> > >> > >> >> > >> > >> >> Best, > >> > >> >> Yang > >> > >> >> > >> > >> >> > >> > >> >> > >> > >> >> > >> > >> >> Andrey Zagrebin <[hidden email]> 于2020年3月4日周三 下午5:34写道: > >> > >> >> > >> > >> >>> Hi All, > >> > >> >>> > >> > >> >>> If you have ever touched the docker topic in Flink, you > >> > >> >>> probably noticed that we have multiple places in docs and repos > >> > which > >> > >> >>> address its various concerns. > >> > >> >>> > >> > >> >>> We have prepared a FLIP [1] to simplify the perception of > docker > >> > topic in > >> > >> >>> Flink by users. It mostly advocates for an approach of > extending > >> > official > >> > >> >>> Flink image from the docker hub. For convenience, it can come > with > >> > a set > >> > >> >>> of > >> > >> >>> bash utilities and documented examples of their usage. The > utilities > >> > >> >>> allow > >> > >> >>> to: > >> > >> >>> > >> > >> >>> - run the docker image in various modes (single job, session > >> > master, > >> > >> >>> task manager etc) > >> > >> >>> - customise the extending Dockerfile > >> > >> >>> - and its entry point > >> > >> >>> > >> > >> >>> Eventually, the FLIP suggests to remove all other user facing > >> > Dockerfiles > >> > >> >>> and building scripts from Flink repo, move all docker docs to > >> > >> >>> apache/flink-docker and adjust existing docker use cases to > refer > >> > to this > >> > >> >>> new approach (mostly Kubernetes now). > >> > >> >>> > >> > >> >>> The first contributed version of Flink docker integration also > >> > contained > >> > >> >>> example and docs for the integration with Bluemix in IBM > cloud. We > >> > also > >> > >> >>> suggest to maintain it outside of Flink repository (cc Markus > >> > Müller). > >> > >> >>> > >> > >> >>> Thanks, > >> > >> >>> Andrey > >> > >> >>> > >> > >> >>> [1] > >> > >> >>> > >> > >> >>> > >> > > https://cwiki.apache.org/confluence/display/FLINK/FLIP-111%3A+Docker+image+unification > >> > >> >>> > >> > >> >> > >> > >> > > >> > >> > -- > >> > >> > > >> > >> > Konstantin Knauf | Head of Product > >> > >> > > >> > >> > +49 160 91394525 > >> > >> > > >> > >> > > >> > >> > Follow us @VervericaData Ververica <https://www.ververica.com/> > >> > >> > > >> > >> > > >> > >> > -- > >> > >> > > >> > >> > Join Flink Forward <https://flink-forward.org/> - The Apache > Flink > >> > >> > Conference > >> > >> > > >> > >> > Stream Processing | Event Driven | Real Time > >> > >> > > >> > >> > -- > >> > >> > > >> > >> > Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany > >> > >> > > >> > >> > -- > >> > >> > Ververica GmbH > >> > >> > Registered at Amtsgericht Charlottenburg: HRB 158244 B > >> > >> > Managing Directors: Timothy Alexander Steinert, Yip Park Tung > Jason, > >> > Ji > >> > >> > (Tony) Cheng > >> > >> > > >> > > |
Thanks for summarising the discussion points, Till.
# Configuration ## Env variables Agree, this looks like an independent effort. ## dynamic program arguments Indeed, jobmanager.sh needs small extension. It can be addressed independently but I think it has chance to be addressed in this release cycle. taskmanager.sh/flink-console.sh/flink-daemon.sh look already fit for this from what I see. On the other hand, we already have FLINK_PROPERTIES and indeed we can keep it until we have env vars. ## FLINK_PROPERTIES I am not really saying to remove this right now. Indeed, it has been already exposed and should stay for the backwards-compatibility at the moment. I was just wondering whether we could maintain/advertise less approaches in future but which are good enough for users. The benefit of env vars / dynamic args approaches is that they are supposed to be already supported by Flink scripts out of the box and would have almost no maintenance for us on the docker side. On the other hand, I can also see that setting FLINK_PROPERTIES may be easier in certain cases comparing to generation of env vars / dynamic args. @Thomas Weise <[hidden email]> The current duplication may be fixed by setting the hardcoded ports after FLINK_PROPERTIES if they are not set. We can look at it during implementation in detail. ## flink_docker_utils configure "option.name" “value” The previously discussed options are mostly for running the official image. This scripted action is mostly for custom Dockerfiles or custom entry point scripts extending the official image where somebody wants to ship a preconfigured custom image. This action would already deduplicate a lot of code for the ports which are set to hardcoded values in the entry point script of the official Dockerfile: if grep -E "^blob\.server\.port:.*" "${CONF_FILE}"; then sed -i -e "s/blob.server.port:*/blob.server.port: 6124/g" "${CONF_FILE}" else echo "blob.server.port: 6124" >> "${CONF_FILE}" fi If we are in doubt to document this for the users and expose as API, we do not have to do it and expose later if needed. I am ok to remove 'flink_docker_utils set_web_ui_port 8081' from FLIP. # Logging ## Logging The idea was to fix the existing console logging properties to log also into files to fix Web UI [1]. Then we can just use Flink scripts with 'start-foreground' argument to log into the stdout and files. ## Stdout/err ### Console I think the stdout/stderr of Flink process might end up in the container stdout/stderr if we run it in 'start-foreground' mode [2] (needs checking). ### Local files The proposal with the *tee* command looks promising. I would prefer to write stdout/err into separate files and preserve them as stdout/err for container logs. This needs more experiments but may be possible with the *tee* command. I suggest to check the details in PRs. # Java/Python/Dev versiona Shipping official images with various versions can be addressed independently. Best, Andrey [1] https://cwiki.apache.org/confluence/display/FLINK/FLIP-111%3A+Docker+image+unification#FLIP-111:Dockerimageunification-Logging [2] https://docs.docker.com/config/containers/logging On Wed, Mar 18, 2020 at 2:14 PM Till Rohrmann <[hidden email]> wrote: > Thanks for creating this FLIP Andrey. I like the general idea pretty much. > > I tried to group some of the above-mentioned points in order to give my 50 > cent. > > # Configuration > > How to configure the Flink process seems to be the biggest question. Due > to historical reasons we have a plethora of different ways on how to > configure the Flink Docker image. > > I think the most general approach would be the configuration via > environment variables where one can specify env variables with the form > FLINK_<CONFIG_KEY>=<VALUE> which will overwrite <CONFIG_KEY> with <VALUE> > in the effective configuration. However, this is something which deserves a > separate effort and is out of scope for this proposal. > > The next best thing for configuring Flink (not the Flink process) would be > dynamic program arguments. For this to work, we would need to extend the > jobmanager.sh and taskmanager.sh scripts. I think this is strictly speaking > also an orthogonal task and could happen as a follow up/independently. > > This leaves us with the envsubst and FLINK_PROPERTIES approaches. Even > though I'm not a huge fan of these approaches, I think we should still > support them for backwards compatibility reasons. Once we support > configuration via env variables we should deprecate these methods and > remove them in a subsequent release. > > Given this, I am bit unsure about introducing yet another way > via flink_docker_utils configure "option.name" “value”. I think we should > only offer this option if we are sure that we want to keep it in the future > and that it won't be superceded by the env variables approach. Otherwise it > will only add more maintenance burden. > > Long story short, with the existing configuration options (envsubts, > FLINK_PROPERTIES) we can already configure the Flink process and Flink > itself. Since maintaining backwards compatibility is important, we could > rely on these mechanisms until we have proper env variable configuration > and don't have to introduce a new way to change the configuration. > > # Logging & Stdout/err > > ## Logging > > I think Konstantin is right and we should provide a log4j.properties file > which, per default, specifies the file and console appender. We could add a > special log4j.properties file to apache/flink-docker which we include in > the Dockerfile. > > This approach will give users the most flexibility w/o relying on magic > (e.g. tailing the log files after starting the process in the background). > > ## Stdout/err > > I think for printing the stdout/err output to STDOUT/ERR and to capture it > in a file there are solutions. For example, one could use `program &2>1 | > tee flink-user-taskexecutor.out` to achieve this. > > # Java version > > I agree that it would be nice to also offer a Java 11 Dockerfile. For the > sake of limiting the scope of this proposal I would suggest to do this as a > follow up issue. > > # Dev version > > Tooling to create a Docker image from the current Flink repository is > indeed very nice for development. As Andrey suggested, I think this would > be a good follow up for this proposal. I don't think that Andrey's current > proposal would block any future developments in this direction. > > # Scripts > > At the moment, I would be in favour of placing the Dockerfile scripts > under apache/flink-docker since they belong more to the Dockerfile than to > Flink's binary distribution. If we see that we might be able to reuse them > for the developer Dockerfile, then we can still move them to the Flink > repository. > > I would refrain from offering special commands to set individual > configuration options (e.g., flink_docker_utils set_web_ui_port 8081). It > should be fine enough to do it via flink_docker-utils conifgure rest.port > 8081 if we cannot solve it via the general configuration mechanism. > > Cheers, > Till > > On Wed, Mar 18, 2020 at 6:38 AM Yangze Guo <[hidden email]> wrote: > >> I second Thomas that we can support both Java 8 and 11. >> >> Best, >> Yangze Guo >> >> On Wed, Mar 18, 2020 at 12:12 PM Thomas Weise <[hidden email]> wrote: >> > >> > --> >> > >> > On Mon, Mar 16, 2020 at 1:58 AM Andrey Zagrebin <[hidden email]> >> wrote: >> >> >> >> Thanks for the further feedback Thomas and Yangze. >> >> >> >> > A generic, dynamic configuration mechanism based on environment >> variables >> >> is essential and it is already supported via envsubst and an >> environment >> >> variable that can supply a configuration fragment >> >> >> >> True, we already have this. As I understand this was introduced for >> >> flexibility to template a custom flink-conf.yaml with env vars, put it >> into >> >> the FLINK_PROPERTIES and merge it with the default one. >> >> Could we achieve the same with the dynamic properties >> (-Drpc.port=1234), >> >> passed as image args to run it, instead of FLINK_PROPERTIES? >> >> They could be also parametrised with env vars. This would require >> >> jobmanager.sh to properly propagate them to >> >> the StandaloneSessionClusterEntrypoint though: >> >> >> https://github.com/docker-flink/docker-flink/pull/82#issuecomment-525285552 >> >> cc @Till >> >> This would provide a unified configuration approach. >> >> >> > >> > How would that look like for the various use cases? The k8s operator >> would need to generate the -Dabc .. -Dxyz entry point command instead of >> setting the FLINK_PROPERTIES environment variable? Potentially that >> introduces additional complexity for little gain. Do most deployment >> platforms that support Docker containers handle the command line route >> well? Backward compatibility may also be a concern. >> > >> >> >> >> > On the flip side, attempting to support a fixed subset of >> configuration >> >> options is brittle and will probably lead to compatibility issues down >> the >> >> road >> >> >> >> I agree with it. The idea was to have just some shortcut scripted >> functions >> >> to set options in flink-conf.yaml for a custom Dockerfile or entry >> point >> >> script. >> >> TASK_MANAGER_NUMBER_OF_TASK_SLOTS could be set as a dynamic property of >> >> started JM. >> >> I am not sure how many users depend on it. Maybe we could remove it. >> >> It also looks we already have somewhat unclean state in >> >> the docker-entrypoint.sh where some ports are set the hardcoded values >> >> and then FLINK_PROPERTIES are applied potentially duplicating options >> in >> >> the result flink-conf.yaml. >> > >> > >> > That is indeed possible and duplicate entries from FLINK_PROPERTIES >> prevail. Unfortunately, the special cases you mention were already >> established and the generic mechanism was added later for the k8s operators. >> > >> >> >> >> >> >> I can see some potential usage of env vars as standard entry point >> args but >> >> for purposes related to something which cannot be achieved by passing >> entry >> >> point args, like changing flink-conf.yaml options. Nothing comes into >> my >> >> mind at the moment. It could be some setting specific to the running >> mode >> >> of the entry point. The mode itself can stay the first arg of the entry >> >> point. >> >> >> >> > I would second that it is desirable to support Java 11 >> >> >> >> > Regarding supporting JAVA 11: >> >> > - Not sure if it is necessary to ship JAVA. Maybe we could just >> change >> >> > the base image from openjdk:8-jre to openjdk:11-jre in template >> docker >> >> > file[1]. Correct me if I understand incorrectly. Also, I agree to >> move >> >> > this out of the scope of this FLIP if it indeed takes much extra >> >> > effort. >> >> >> >> This is what I meant by bumping up the Java version in the docker hub >> Flink >> >> image: >> >> FROM openjdk:8-jre -> FROM openjdk:11-jre >> >> This can be polled dependently in user mailing list. >> > >> > >> > That sounds reasonable as long as we can still support both Java >> versions (i.e. provide separate images for 8 and 11). >> > >> >> >> >> >> >> > and in general use a base image that allows the (straightforward) >> use of >> >> more recent versions of other software (Python etc.) >> >> >> >> This can be polled whether to always include some version of python >> into >> >> the docker hub image. >> >> A potential problem here is once it is there, it is some hassle to >> >> remove/change it in a custom extended Dockerfile. >> >> >> >> It would be also nice to avoid maintaining images for various >> combinations >> >> of installed Java/Scala/Python in docker hub. >> >> >> >> > Regarding building from local dist: >> >> > - Yes, I bring this up mostly for development purpose. Since k8s is >> >> > popular, I believe more and more developers would like to test their >> >> > work on k8s cluster. I'm not sure should all developers write a >> custom >> >> > docker file themselves in this scenario. Thus, I still prefer to >> >> > provide a script for devs. >> >> > - I agree to keep the scope of this FLIP mostly for those normal >> >> > users. But as far as I can see, supporting building from local dist >> >> > would not take much extra effort. >> >> > - The maven docker plugin sounds good. I'll take a look at it. >> >> >> >> I would see any scripts introduced in this FLIP also as potential >> building >> >> blocks for a custom dev Dockerfile. >> >> Maybe, this will be all what we need for dev images or we write a dev >> >> Dockerfile, highly parametrised for building a dev image. >> >> If scripts stay in apache/flink-docker, it is also somewhat >> inconvenient to >> >> use them in the main Flink repo but possible. >> >> If we move them to apache/flink then we will have to e.g. include them >> into >> >> the release to make them easily available in apache/flink-docker and >> >> maintain them in main repo, although they are only docker specific. >> >> All in all, I would say, once we implement them, we can revisit this >> topic. >> >> >> >> Best, >> >> Andrey >> >> >> >> On Wed, Mar 11, 2020 at 8:58 AM Yangze Guo <[hidden email]> wrote: >> >> >> >> > Thanks for the reply, Andrey. >> >> > >> >> > Regarding building from local dist: >> >> > - Yes, I bring this up mostly for development purpose. Since k8s is >> >> > popular, I believe more and more developers would like to test their >> >> > work on k8s cluster. I'm not sure should all developers write a >> custom >> >> > docker file themselves in this scenario. Thus, I still prefer to >> >> > provide a script for devs. >> >> > - I agree to keep the scope of this FLIP mostly for those normal >> >> > users. But as far as I can see, supporting building from local dist >> >> > would not take much extra effort. >> >> > - The maven docker plugin sounds good. I'll take a look at it. >> >> > >> >> > Regarding supporting JAVA 11: >> >> > - Not sure if it is necessary to ship JAVA. Maybe we could just >> change >> >> > the base image from openjdk:8-jre to openjdk:11-jre in template >> docker >> >> > file[1]. Correct me if I understand incorrectly. Also, I agree to >> move >> >> > this out of the scope of this FLIP if it indeed takes much extra >> >> > effort. >> >> > >> >> > Regarding the custom configuration, the mechanism that Thomas >> mentioned >> >> > LGTM. >> >> > >> >> > [1] >> >> > >> https://github.com/apache/flink-docker/blob/master/Dockerfile-debian.template >> >> > >> >> > Best, >> >> > Yangze Guo >> >> > >> >> > On Wed, Mar 11, 2020 at 5:52 AM Thomas Weise <[hidden email]> wrote: >> >> > > >> >> > > Thanks for working on improvements to the Flink Docker container >> images. >> >> > This will be important as more and more users are looking to adopt >> >> > Kubernetes and other deployment tooling that relies on Docker images. >> >> > > >> >> > > A generic, dynamic configuration mechanism based on environment >> >> > variables is essential and it is already supported via envsubst and >> an >> >> > environment variable that can supply a configuration fragment: >> >> > > >> >> > > >> >> > >> https://github.com/apache/flink-docker/blob/09adf2dcd99abfb6180e1e2b5b917b288e0c01f6/docker-entrypoint.sh#L88 >> >> > > >> >> > >> https://github.com/apache/flink-docker/blob/09adf2dcd99abfb6180e1e2b5b917b288e0c01f6/docker-entrypoint.sh#L85 >> >> > > >> >> > > This gives the necessary control for infrastructure use cases that >> aim >> >> > to supply deployment tooling other users. An example in this >> category this >> >> > is the FlinkK8sOperator: >> >> > > >> >> > > >> https://github.com/lyft/flinkk8soperator/tree/master/examples/wordcount >> >> > > >> >> > > On the flip side, attempting to support a fixed subset of >> configuration >> >> > options is brittle and will probably lead to compatibility issues >> down the >> >> > road: >> >> > > >> >> > > >> >> > >> https://github.com/apache/flink-docker/blob/09adf2dcd99abfb6180e1e2b5b917b288e0c01f6/docker-entrypoint.sh#L97 >> >> > > >> >> > > Besides the configuration, it may be worthwhile to see in which >> other >> >> > ways the base Docker images can provide more flexibility to >> incentivize >> >> > wider adoption. >> >> > > >> >> > > I would second that it is desirable to support Java 11 and in >> general >> >> > use a base image that allows the (straightforward) use of more recent >> >> > versions of other software (Python etc.) >> >> > > >> >> > > >> >> > >> https://github.com/apache/flink-docker/blob/d3416e720377e9b4c07a2d0f4591965264ac74c5/Dockerfile-debian.template#L19 >> >> > > >> >> > > Thanks, >> >> > > Thomas >> >> > > >> >> > > On Tue, Mar 10, 2020 at 12:26 PM Andrey Zagrebin < >> [hidden email]> >> >> > wrote: >> >> > >> >> >> > >> Hi All, >> >> > >> >> >> > >> Thanks a lot for the feedback! >> >> > >> >> >> > >> *@Yangze Guo* >> >> > >> >> >> > >> - Regarding the flink_docker_utils#install_flink function, I >> think it >> >> > >> > should also support build from local dist and build from a >> >> > >> > user-defined archive. >> >> > >> >> >> > >> I suppose you bring this up mostly for development purpose or >> powerful >> >> > >> users. >> >> > >> Most of normal users are usually interested in mainstream released >> >> > versions >> >> > >> of Flink. >> >> > >> Although, you are bring a valid concern, my idea was to keep >> scope of >> >> > this >> >> > >> FLIP mostly for those normal users. >> >> > >> The powerful users are usually capable to design a completely >> >> > >> custom Dockerfile themselves. >> >> > >> At the moment, we already have custom Dockerfiles e.g. for tests >> in >> >> > >> >> >> > >> flink-end-to-end-tests/test-scripts/docker-hadoop-secure-cluster/Dockerfile. >> >> > >> We can add something similar for development purposes and maybe >> >> > introduce a >> >> > >> special maven goal. There is a maven docker plugin, afaik. >> >> > >> I will add this to FLIP as next step. >> >> > >> >> >> > >> - It seems that the install_shaded_hadoop could be an option of >> >> > >> > install_flink >> >> > >> >> >> > >> I woud rather think about this as a separate independent optional >> step. >> >> > >> >> >> > >> - Should we support JAVA 11? Currently, most of the docker file >> based on >> >> > >> > JAVA 8. >> >> > >> >> >> > >> Indeed, it is a valid concern. Java version is a fundamental >> property of >> >> > >> the docker image. >> >> > >> To customise this in the current mainstream image is difficult, >> this >> >> > would >> >> > >> require to ship it w/o Java at all. >> >> > >> Or this is a separate discussion whether we want to distribute >> docker >> >> > hub >> >> > >> images with different Java versions or just bump it to Java 11. >> >> > >> This should be easy in a custom Dockerfile for development >> purposes >> >> > though >> >> > >> as mentioned before. >> >> > >> >> >> > >> - I do not understand how to set config options through >> >> > >> >> >> > >> "flink_docker_utils configure"? Does this step happen during the >> image >> >> > >> > build or the container start? If it happens during the image >> build, >> >> > >> > there would be a new image every time we change the config. If >> it just >> >> > >> > a part of the container entrypoint, I think there is no need to >> add a >> >> > >> > configure command, we could just add all dynamic config options >> to the >> >> > >> > args list of "start_jobmaster"/"start_session_jobmanager". Am I >> >> > >> > understanding this correctly? >> >> > >> >> >> > >> `flink_docker_utils configure ...` can be called everywhere: >> >> > >> - while building a custom image (`RUN flink_docker_utils >> configure ..`) >> >> > by >> >> > >> extending our base image from docker hub (`from flink`) >> >> > >> - in a custom entry point as well >> >> > >> I will check this but if user can also pass a dynamic config >> option it >> >> > also >> >> > >> sounds like a good option >> >> > >> Our standard entry point script in base image could just properly >> >> > forward >> >> > >> the arguments to the Flink process. >> >> > >> >> >> > >> @Yang Wang >> >> > >> >> >> > >> > About docker utils >> >> > >> > I really like the idea to provide some utils for the docker >> file and >> >> > entry >> >> > >> > point. The >> >> > >> > `flink_docker_utils` will help to build the image easier. I am >> not >> >> > sure >> >> > >> > about the >> >> > >> > `flink_docker_utils start_jobmaster`. Do you mean when we build >> a >> >> > docker >> >> > >> > image, we >> >> > >> > need to add `RUN flink_docker_utils start_jobmaster` in the >> docker >> >> > file? >> >> > >> > Why do we need this? >> >> > >> >> >> > >> This is a scripted action to start JM. It can be called >> everywhere. >> >> > >> Indeed, it does not make too much sense to run it in Dockerfile. >> >> > >> Mostly, the idea was to use in a custom entry point. When our base >> >> > docker >> >> > >> hub image is started its entry point can be also completely >> overridden. >> >> > >> The actions are also sorted in the FLIP: for Dockerfile or for >> entry >> >> > point. >> >> > >> E.g. our standard entry point script in the base docker hub image >> can >> >> > >> already use it. >> >> > >> Anyways, it was just an example, the details are to be defined in >> Jira, >> >> > imo. >> >> > >> >> >> > >> > About docker entry point >> >> > >> > I agree with you that the docker entry point could more >> powerful with >> >> > more >> >> > >> > functionality. >> >> > >> > Mostly, it is about to override the config options. If we >> support >> >> > dynamic >> >> > >> > properties, i think >> >> > >> > it is more convenient for users without any learning curve. >> >> > >> > `docker run flink session_jobmanager -D rest.bind-port=8081` >> >> > >> >> >> > >> Indeed, as mentioned before, it can be a better option. >> >> > >> The standard entry point also decides at least what to run JM or >> TM. I >> >> > >> think we will see what else makes sense to include there during >> the >> >> > >> implementation. >> >> > >> Some specifics may be more convenient to set with env vars as >> Konstantin >> >> > >> mentioned. >> >> > >> >> >> > >> > About the logging >> >> > >> > Updating the `log4j-console.properties` to support multiple >> appender >> >> > is a >> >> > >> > better option. >> >> > >> > Currently, the native K8s is suggesting users to debug the logs >> in >> >> > this >> >> > >> > way[1]. However, >> >> > >> > there is also some problems. The stderr and stdout of JM/TM >> processes >> >> > could >> >> > >> > not be >> >> > >> > forwarded to the docker container console. >> >> > >> >> >> > >> Strange, we should check maybe there is a docker option to query >> the >> >> > >> container's stderr output as well. >> >> > >> If we forward Flink process stdout as usual in bash console, it >> should >> >> > not >> >> > >> be a problem. Why can it not be forwarded? >> >> > >> >> >> > >> @Konstantin Knauf >> >> > >> >> >> > >> For the entrypoint, have you considered to also allow setting >> >> > configuration >> >> > >> > via environment variables as in "docker run -e >> >> > FLINK_REST_BIN_PORT=8081 >> >> > >> > ..."? This is quite common and more flexible, e.g. it makes it >> very >> >> > easy to >> >> > >> > pass values of Kubernetes Secrets into the Flink configuration. >> >> > >> >> >> > >> This is indeed an interesting option to pass arguments to the >> entry >> >> > point >> >> > >> in general. >> >> > >> For the config options, the dynamic args can be a better option as >> >> > >> mentioned above. >> >> > >> >> >> > >> With respect to logging, I would opt to keep this very basic and >> to only >> >> > >> > support logging to the console (maybe with a fix for the web >> user >> >> > >> > interface). For everything else, users can easily build their >> own >> >> > images >> >> > >> > based on library/flink (provide the dependencies, change the >> logging >> >> > >> > configuration). >> >> > >> >> >> > >> agree >> >> > >> >> >> > >> Thanks, >> >> > >> Andrey >> >> > >> >> >> > >> On Sun, Mar 8, 2020 at 8:55 PM Konstantin Knauf < >> >> > [hidden email]> >> >> > >> wrote: >> >> > >> >> >> > >> > Hi Andrey, >> >> > >> > >> >> > >> > thanks a lot for this proposal. The variety of Docker files in >> the >> >> > project >> >> > >> > has been causing quite some confusion. >> >> > >> > >> >> > >> > For the entrypoint, have you considered to also allow setting >> >> > >> > configuration via environment variables as in "docker run -e >> >> > >> > FLINK_REST_BIN_PORT=8081 ..."? This is quite common and more >> >> > flexible, e.g. >> >> > >> > it makes it very easy to pass values of Kubernetes Secrets into >> the >> >> > Flink >> >> > >> > configuration. >> >> > >> > >> >> > >> > With respect to logging, I would opt to keep this very basic >> and to >> >> > only >> >> > >> > support logging to the console (maybe with a fix for the web >> user >> >> > >> > interface). For everything else, users can easily build their >> own >> >> > images >> >> > >> > based on library/flink (provide the dependencies, change the >> logging >> >> > >> > configuration). >> >> > >> > >> >> > >> > Cheers, >> >> > >> > >> >> > >> > Konstantin >> >> > >> > >> >> > >> > >> >> > >> > On Thu, Mar 5, 2020 at 11:01 AM Yang Wang < >> [hidden email]> >> >> > wrote: >> >> > >> > >> >> > >> >> Hi Andrey, >> >> > >> >> >> >> > >> >> >> >> > >> >> Thanks for driving this significant FLIP. From the user ML, we >> could >> >> > also >> >> > >> >> know there are >> >> > >> >> many users running Flink in container environment. Then the >> docker >> >> > image >> >> > >> >> will be the >> >> > >> >> very basic requirement. Just as you say, we should provide a >> unified >> >> > >> >> place for all various >> >> > >> >> usage(e.g. session, job, native k8s, swarm, etc.). >> >> > >> >> >> >> > >> >> >> >> > >> >> > About docker utils >> >> > >> >> >> >> > >> >> I really like the idea to provide some utils for the docker >> file and >> >> > >> >> entry point. The >> >> > >> >> `flink_docker_utils` will help to build the image easier. I am >> not >> >> > sure >> >> > >> >> about the >> >> > >> >> `flink_docker_utils start_jobmaster`. Do you mean when we >> build a >> >> > docker >> >> > >> >> image, we >> >> > >> >> need to add `RUN flink_docker_utils start_jobmaster` in the >> docker >> >> > file? >> >> > >> >> Why do we need this? >> >> > >> >> >> >> > >> >> >> >> > >> >> > About docker entry point >> >> > >> >> >> >> > >> >> I agree with you that the docker entry point could more >> powerful with >> >> > >> >> more functionality. >> >> > >> >> Mostly, it is about to override the config options. If we >> support >> >> > dynamic >> >> > >> >> properties, i think >> >> > >> >> it is more convenient for users without any learning curve. >> >> > >> >> `docker run flink session_jobmanager -D rest.bind-port=8081` >> >> > >> >> >> >> > >> >> >> >> > >> >> > About the logging >> >> > >> >> >> >> > >> >> Updating the `log4j-console.properties` to support multiple >> appender >> >> > is a >> >> > >> >> better option. >> >> > >> >> Currently, the native K8s is suggesting users to debug the >> logs in >> >> > this >> >> > >> >> way[1]. However, >> >> > >> >> there is also some problems. The stderr and stdout of JM/TM >> processes >> >> > >> >> could not be >> >> > >> >> forwarded to the docker container console. >> >> > >> >> >> >> > >> >> >> >> > >> >> [1]. >> >> > >> >> >> >> > >> https://ci.apache.org/projects/flink/flink-docs-master/ops/deployment/native_kubernetes.html#log-files >> >> > >> >> >> >> > >> >> >> >> > >> >> Best, >> >> > >> >> Yang >> >> > >> >> >> >> > >> >> >> >> > >> >> >> >> > >> >> >> >> > >> >> Andrey Zagrebin <[hidden email]> 于2020年3月4日周三 下午5:34写道: >> >> > >> >> >> >> > >> >>> Hi All, >> >> > >> >>> >> >> > >> >>> If you have ever touched the docker topic in Flink, you >> >> > >> >>> probably noticed that we have multiple places in docs and >> repos >> >> > which >> >> > >> >>> address its various concerns. >> >> > >> >>> >> >> > >> >>> We have prepared a FLIP [1] to simplify the perception of >> docker >> >> > topic in >> >> > >> >>> Flink by users. It mostly advocates for an approach of >> extending >> >> > official >> >> > >> >>> Flink image from the docker hub. For convenience, it can come >> with >> >> > a set >> >> > >> >>> of >> >> > >> >>> bash utilities and documented examples of their usage. The >> utilities >> >> > >> >>> allow >> >> > >> >>> to: >> >> > >> >>> >> >> > >> >>> - run the docker image in various modes (single job, >> session >> >> > master, >> >> > >> >>> task manager etc) >> >> > >> >>> - customise the extending Dockerfile >> >> > >> >>> - and its entry point >> >> > >> >>> >> >> > >> >>> Eventually, the FLIP suggests to remove all other user facing >> >> > Dockerfiles >> >> > >> >>> and building scripts from Flink repo, move all docker docs to >> >> > >> >>> apache/flink-docker and adjust existing docker use cases to >> refer >> >> > to this >> >> > >> >>> new approach (mostly Kubernetes now). >> >> > >> >>> >> >> > >> >>> The first contributed version of Flink docker integration also >> >> > contained >> >> > >> >>> example and docs for the integration with Bluemix in IBM >> cloud. We >> >> > also >> >> > >> >>> suggest to maintain it outside of Flink repository (cc Markus >> >> > Müller). >> >> > >> >>> >> >> > >> >>> Thanks, >> >> > >> >>> Andrey >> >> > >> >>> >> >> > >> >>> [1] >> >> > >> >>> >> >> > >> >>> >> >> > >> https://cwiki.apache.org/confluence/display/FLINK/FLIP-111%3A+Docker+image+unification >> >> > >> >>> >> >> > >> >> >> >> > >> > >> >> > >> > -- >> >> > >> > >> >> > >> > Konstantin Knauf | Head of Product >> >> > >> > >> >> > >> > +49 160 91394525 >> >> > >> > >> >> > >> > >> >> > >> > Follow us @VervericaData Ververica <https://www.ververica.com/> >> >> > >> > >> >> > >> > >> >> > >> > -- >> >> > >> > >> >> > >> > Join Flink Forward <https://flink-forward.org/> - The Apache >> Flink >> >> > >> > Conference >> >> > >> > >> >> > >> > Stream Processing | Event Driven | Real Time >> >> > >> > >> >> > >> > -- >> >> > >> > >> >> > >> > Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany >> >> > >> > >> >> > >> > -- >> >> > >> > Ververica GmbH >> >> > >> > Registered at Amtsgericht Charlottenburg: HRB 158244 B >> >> > >> > Managing Directors: Timothy Alexander Steinert, Yip Park Tung >> Jason, >> >> > Ji >> >> > >> > (Tony) Cheng >> >> > >> > >> >> > >> > |
In reply to this post by Andrey Zagrebin-4
Thanks Andrey for working on this, and everyone else for your feedback.
This FLIP inspired me to discuss and write down some ideas I've had for a while about configuring and running Flink (especially in Docker) that go beyond the scope of this FLIP, but don't contradict what it sets out to do. The crux of it is that Flink should be maximally configurable using environment variables, and not require manipulation of the filesystem (i.e. moving/linking JARs or editing config files) in order to run in a large majority of cases. And beyond that, particular for running Flink in Docker, is that as much logic as possible should be a part of Flink itself and not, for instance, in the docker-entrypoint.sh script. I've resisted adding additional logic to the Flink Docker images except where necessary since the beginning, and I believe we can get to the point where the only thing the entrypoint script does is drop privileges before invoking a script included in Flink. Ultimately, my ideal end-goal for running Flink in containers would fulfill > the following points: > > - A user can configure all “start-time” aspects of Flink with > environment variables, including additions to the classpath > - Flink automatically adapts to the resources available to the > container (such as what BashJavaUtils helps with today) > - A user can include additional JARs using a mounted volume, or at > image build time with convenient tooling > - The role/mode (jobmanager, session) is specified as a command line > argument, with a single entrypoint program sufficing for all uses of the > image > > As a bonus, if we could eliminate some or most of the layers of shell > scripts that are involved in starting a Flink server, perhaps by > re-implementing this part of the stack in Java, and exec-ing to actually > run Flink with the proper java CLI arguments, I think it would be a big win > for the project. You can read the rest of my notes here: https://docs.google.com/document/d/1JCACSeDaqeZiXD9G1XxQBunwi-chwrdnFm38U1JxTDQ/edit On Wed, Mar 4, 2020 at 10:34 AM Andrey Zagrebin <[hidden email]> wrote: > Hi All, > > If you have ever touched the docker topic in Flink, you > probably noticed that we have multiple places in docs and repos which > address its various concerns. > > We have prepared a FLIP [1] to simplify the perception of docker topic in > Flink by users. It mostly advocates for an approach of extending official > Flink image from the docker hub. For convenience, it can come with a set of > bash utilities and documented examples of their usage. The utilities allow > to: > > - run the docker image in various modes (single job, session master, > task manager etc) > - customise the extending Dockerfile > - and its entry point > > Eventually, the FLIP suggests to remove all other user facing Dockerfiles > and building scripts from Flink repo, move all docker docs to > apache/flink-docker and adjust existing docker use cases to refer to this > new approach (mostly Kubernetes now). > > The first contributed version of Flink docker integration also contained > example and docs for the integration with Bluemix in IBM cloud. We also > suggest to maintain it outside of Flink repository (cc Markus Müller). > > Thanks, > Andrey > > [1] > https://cwiki.apache.org/confluence/display/FLINK/FLIP-111%3A+Docker+image+unification > |
Hey all,
thanks for the proposal and the detailed discussion. In particular, thanks to Andrey for starting this thread and to Patrick for the additional ideas in the linked Google doc. I find many of the improvements proposed during the discussion (such as the unified entrypoint in Flink, proper configuration via environment variables, Dockerfiles for development, etc.) really important. At the same time, I believe that these improvements have quite a large scope and could be tackled independently as Till already suggested. I think we should ideally split the discussions for those improvements out of this thread and focus on the main target of FLIP-111. To me the major point of this FLIP is to consolidate existing Dockerfiles into apache/flink-docker and document typical usage scenarios (e.g. linking plugins, installing shaded Hadoop, running a job cluster, etc.). In order to achieve this, I think we could move forward as follows: (1) Extend the entrypoint script in apache/flink-docker to start the job cluster entry point => this is currently missing and would block removal of the Dockerfile in flink-container (2) Extend the example log4j-console configuration => support log retrieval from the Flink UI out of the box (3) Document typical usage scenarios in apache/flink-docker => this should replace the proposed flink_docker_utils helper (4) Remove the existing Dockerfiles from apache/flink I really like the convenience of a script such as flink_docker_utils, but I think we should avoid it for now, because most of the desired usage scenarios can be covered by documentation. After we have concluded (1)-(4) we can take a holistic look and identify what would benefit the most from such a script and how it would interact with the other planned improvements. I think this will give us a good basis to tackle the other major improvements that were proposed. – Ufuk On Thu, Apr 2, 2020 at 4:34 PM Patrick Lucas <[hidden email]> wrote: > > Thanks Andrey for working on this, and everyone else for your feedback. > > This FLIP inspired me to discuss and write down some ideas I've had for a > while about configuring and running Flink (especially in Docker) that go > beyond the scope of this FLIP, but don't contradict what it sets out to do. > > The crux of it is that Flink should be maximally configurable using > environment variables, and not require manipulation of the filesystem (i.e. > moving/linking JARs or editing config files) in order to run in a large > majority of cases. And beyond that, particular for running Flink in Docker, > is that as much logic as possible should be a part of Flink itself and not, > for instance, in the docker-entrypoint.sh script. I've resisted adding > additional logic to the Flink Docker images except where necessary since > the beginning, and I believe we can get to the point where the only thing > the entrypoint script does is drop privileges before invoking a script > included in Flink. > > Ultimately, my ideal end-goal for running Flink in containers would fulfill > > the following points: > > > > - A user can configure all “start-time” aspects of Flink with > > environment variables, including additions to the classpath > > - Flink automatically adapts to the resources available to the > > container (such as what BashJavaUtils helps with today) > > - A user can include additional JARs using a mounted volume, or at > > image build time with convenient tooling > > - The role/mode (jobmanager, session) is specified as a command line > > argument, with a single entrypoint program sufficing for all uses of > > image > > > > As a bonus, if we could eliminate some or most of the layers of shell > > scripts that are involved in starting a Flink server, perhaps by > > re-implementing this part of the stack in Java, and exec-ing to actually > > run Flink with the proper java CLI arguments, I think it would be a big win > > for the project. > > > You can read the rest of my notes here: > https://docs.google.com/document/d/1JCACSeDaqeZiXD9G1XxQBunwi-chwrdnFm38U1JxTDQ/edit > > On Wed, Mar 4, 2020 at 10:34 AM Andrey Zagrebin <[hidden email]> > wrote: > > > Hi All, > > > > If you have ever touched the docker topic in Flink, you > > probably noticed that we have multiple places in docs and repos which > > address its various concerns. > > > > We have prepared a FLIP [1] to simplify the perception of docker topic > > Flink by users. It mostly advocates for an approach of extending official > > Flink image from the docker hub. For convenience, it can come with a set of > > bash utilities and documented examples of their usage. The utilities allow > > to: > > > > - run the docker image in various modes (single job, session master, > > task manager etc) > > - customise the extending Dockerfile > > - and its entry point > > > > Eventually, the FLIP suggests to remove all other user facing Dockerfiles > > and building scripts from Flink repo, move all docker docs to > > apache/flink-docker and adjust existing docker use cases to refer to this > > new approach (mostly Kubernetes now). > > > > The first contributed version of Flink docker integration also contained > > example and docs for the integration with Bluemix in IBM cloud. We also > > suggest to maintain it outside of Flink repository (cc Markus Müller). > > > > Thanks, > > Andrey > > > > [1] > > > > |
Hi Ufuk,
Thanks for make the conclusion and directly point out what need to be done in FLIP-111. I agree with you that we should narrow down the scope and focus the most important and basic part about docker image unification. (1) Extend the entrypoint script in apache/flink-docker to start the job > cluster entry point I want to add a small requirement for the entry point script. Currently, for the native K8s integration, we are using the apache/flink-docker image, but with different entry point("kubernetes-entry.sh"). Generate the java cmd in KubernetesUtils and run it in the entry point. I really hope it could merge to apache/flink-docker "docker-entrypoint.sh". (2) Extend the example log4j-console configuration > => support log retrieval from the Flink UI out of the box If you mean to update the "flink-dist/conf/log4j-console.properties" to support console and local log files. I will say "+1". But we need to find a proper way to make stdout/stderr output both available for console and log files. Maybe till's proposal could help to solve this. "`program &2>1 | tee flink-user-taskexecutor.out`" (3) Document typical usage scenarios in apache/flink-docker > => this should replace the proposed flink_docker_utils helper I agree with you that in the first step, the documentation is enough for typical usage(e.g. standalone session, standalone perjob, native, plugins, python, etc.). Best, Yang Ufuk Celebi <[hidden email]> 于2020年4月3日周五 上午1:03写道: > Hey all, > > thanks for the proposal and the detailed discussion. In particular, thanks > to Andrey for starting this thread and to Patrick for the additional ideas > in the linked Google doc. > > I find many of the improvements proposed during the discussion (such as the > unified entrypoint in Flink, proper configuration via environment > variables, Dockerfiles for development, etc.) really important. At the same > time, I believe that these improvements have quite a large scope and could > be tackled independently as Till already suggested. I think we should > ideally split the discussions for those improvements out of this thread and > focus on the main target of FLIP-111. > > To me the major point of this FLIP is to consolidate existing Dockerfiles > into apache/flink-docker and document typical usage scenarios (e.g. linking > plugins, installing shaded Hadoop, running a job cluster, etc.). > > In order to achieve this, I think we could move forward as follows: > > (1) Extend the entrypoint script in apache/flink-docker to start the job > cluster entry point > => this is currently missing and would block removal of the Dockerfile in > flink-container > > (2) Extend the example log4j-console configuration > => support log retrieval from the Flink UI out of the box > > (3) Document typical usage scenarios in apache/flink-docker > => this should replace the proposed flink_docker_utils helper > > (4) Remove the existing Dockerfiles from apache/flink > > > I really like the convenience of a script such as flink_docker_utils, but I > think we should avoid it for now, because most of the desired usage > scenarios can be covered by documentation. After we have concluded (1)-(4) > we can take a holistic look and identify what would benefit the most from > such a script and how it would interact with the other planned > improvements. > > I think this will give us a good basis to tackle the other major > improvements that were proposed. > > – Ufuk > > On Thu, Apr 2, 2020 at 4:34 PM Patrick Lucas <[hidden email]> > wrote: > > > > Thanks Andrey for working on this, and everyone else for your feedback. > > > > This FLIP inspired me to discuss and write down some ideas I've had for a > > while about configuring and running Flink (especially in Docker) that go > > beyond the scope of this FLIP, but don't contradict what it sets out to > do. > > > > The crux of it is that Flink should be maximally configurable using > > environment variables, and not require manipulation of the filesystem > (i.e. > > moving/linking JARs or editing config files) in order to run in a large > > majority of cases. And beyond that, particular for running Flink in > Docker, > > is that as much logic as possible should be a part of Flink itself and > not, > > for instance, in the docker-entrypoint.sh script. I've resisted adding > > additional logic to the Flink Docker images except where necessary since > > the beginning, and I believe we can get to the point where the only thing > > the entrypoint script does is drop privileges before invoking a script > > included in Flink. > > > > Ultimately, my ideal end-goal for running Flink in containers would > fulfill > > > the following points: > > > > > > - A user can configure all “start-time” aspects of Flink with > > > environment variables, including additions to the classpath > > > - Flink automatically adapts to the resources available to the > > > container (such as what BashJavaUtils helps with today) > > > - A user can include additional JARs using a mounted volume, or at > > > image build time with convenient tooling > > > - The role/mode (jobmanager, session) is specified as a command line > > > argument, with a single entrypoint program sufficing for all uses of > the > > > image > > > > > > As a bonus, if we could eliminate some or most of the layers of shell > > > scripts that are involved in starting a Flink server, perhaps by > > > re-implementing this part of the stack in Java, and exec-ing to > actually > > > run Flink with the proper java CLI arguments, I think it would be a big > win > > > for the project. > > > > > > You can read the rest of my notes here: > > > > https://docs.google.com/document/d/1JCACSeDaqeZiXD9G1XxQBunwi-chwrdnFm38U1JxTDQ/edit > > > > On Wed, Mar 4, 2020 at 10:34 AM Andrey Zagrebin <[hidden email]> > > wrote: > > > > > Hi All, > > > > > > If you have ever touched the docker topic in Flink, you > > > probably noticed that we have multiple places in docs and repos which > > > address its various concerns. > > > > > > We have prepared a FLIP [1] to simplify the perception of docker topic > in > > > Flink by users. It mostly advocates for an approach of extending > official > > > Flink image from the docker hub. For convenience, it can come with a > set of > > > bash utilities and documented examples of their usage. The utilities > allow > > > to: > > > > > > - run the docker image in various modes (single job, session master, > > > task manager etc) > > > - customise the extending Dockerfile > > > - and its entry point > > > > > > Eventually, the FLIP suggests to remove all other user facing > Dockerfiles > > > and building scripts from Flink repo, move all docker docs to > > > apache/flink-docker and adjust existing docker use cases to refer to > this > > > new approach (mostly Kubernetes now). > > > > > > The first contributed version of Flink docker integration also > contained > > > example and docs for the integration with Bluemix in IBM cloud. We also > > > suggest to maintain it outside of Flink repository (cc Markus Müller). > > > > > > Thanks, > > > Andrey > > > > > > [1] > > > > > https://cwiki.apache.org/confluence/display/FLINK/FLIP-111%3A+Docker+image+unification > > > > |
Hey Yang,
thanks! See inline answers. On Fri, Apr 3, 2020 at 5:11 AM Yang Wang <[hidden email]> wrote: > Hi Ufuk, > > Thanks for make the conclusion and directly point out what need to be done > in > FLIP-111. I agree with you that we should narrow down the scope and focus > the > most important and basic part about docker image unification. > > (1) Extend the entrypoint script in apache/flink-docker to start the job >> cluster entry point > > I want to add a small requirement for the entry point script. Currently, > for the native > K8s integration, we are using the apache/flink-docker image, but with > different entry > point("kubernetes-entry.sh"). Generate the java cmd in KubernetesUtils and > run it > in the entry point. I really hope it could merge to apache/flink-docker > "docker-entrypoint.sh". > The script [1] only adds the FLINK_CLASSPATH env var which seems generally reasonable to me. But since principled classpath and entrypoint configuration is somewhat related to the follow-up improvement proposals, I could also see this being done after FLIP-111. > (2) Extend the example log4j-console configuration >> => support log retrieval from the Flink UI out of the box > > If you mean to update the "flink-dist/conf/log4j-console.properties" to > support console and > local log files. I will say "+1". But we need to find a proper way to make > stdout/stderr output > both available for console and log files. Maybe till's proposal could help > to solve this. > "`program &2>1 | tee flink-user-taskexecutor.out`" > I think we can simply add a rolling file appender with a limit on the log size. – Ufuk [1] https://github.com/apache/flink/blob/master/flink-dist/src/main/flink-bin/kubernetes-bin/kubernetes-entry.sh |
Hi everyone,
just a small inline comment. On Fri, Apr 3, 2020 at 11:42 AM Ufuk Celebi <[hidden email]> wrote: > Hey Yang, > > thanks! See inline answers. > > On Fri, Apr 3, 2020 at 5:11 AM Yang Wang <[hidden email]> wrote: > > > Hi Ufuk, > > > > Thanks for make the conclusion and directly point out what need to be > done > > in > > FLIP-111. I agree with you that we should narrow down the scope and focus > > the > > most important and basic part about docker image unification. > > > > (1) Extend the entrypoint script in apache/flink-docker to start the job > >> cluster entry point > > > > I want to add a small requirement for the entry point script. Currently, > > for the native > > K8s integration, we are using the apache/flink-docker image, but with > > different entry > > point("kubernetes-entry.sh"). Generate the java cmd in KubernetesUtils > and > > run it > > in the entry point. I really hope it could merge to apache/flink-docker > > "docker-entrypoint.sh". > > > > The script [1] only adds the FLINK_CLASSPATH env var which seems generally > reasonable to me. But since principled classpath and entrypoint > configuration is somewhat related to the follow-up improvement proposals, I > could also see this being done after FLIP-111. > > > > (2) Extend the example log4j-console configuration > >> => support log retrieval from the Flink UI out of the box > > > > If you mean to update the "flink-dist/conf/log4j-console.properties" to > > support console and > > local log files. I will say "+1". But we need to find a proper way to > make > > stdout/stderr output > > both available for console and log files. Maybe till's proposal could > help > > to solve this. > > "`program &2>1 | tee flink-user-taskexecutor.out`" > > > > I think we can simply add a rolling file appender with a limit on the log > size. > > I think this won't solve Yang's concern. What he wants to achieve is that *.err file which are accessible from the web ui. I don't think that log appender will help with this problem. Cheers, Till > – Ufuk > > [1] > > https://github.com/apache/flink/blob/master/flink-dist/src/main/flink-bin/kubernetes-bin/kubernetes-entry.sh > |
Hi everyone,
Patrick and Ufuk, thanks a lot for more ideas and suggestions! I have updated the FLIP according to the current state of discussion. Now it also contains the implementation steps and future follow-ups. Please, review if there are any concerns. The order of the steps aims for keeping Flink releasable at any point if something does not have enough time to get in. It looks that we are reaching mostly a consensus for the open questions. There is also a list of items, which have been discussed in this thread, and short summary below. As soon as there are no concerns, I will create a voting thread. I also added some thoughts for further customising logging setup. This may be an optional follow-up which is additional to the default logging into files for Web UI. # FLIP scope The focus is users of the official releases. Create docs for how to use the official docker image. Remove other Dockerfiles in Flink repo. Rely on running the official docker image in different modes (JM/TM). Customise running the official image with env vars (This should minimise manual manipulating of local files and creation of a custom image). # Base oficial image ## Java versions There is a separate effort for this: https://github.com/apache/flink-docker/pull/9 # Run image ## Entry point modes JM session, JM job, TM ## Entry point config We use env vars for this, e.g. FLINK_PROPERTIES and ENABLE_BUILT_IN_PLUGINS ## Flink config options We document the existing FLINK_PROPERTIES env var to override config options in flink-conf.yaml. Then later, we do not need to expose and handle any other special env vars for config options (address, port etc). The future plan is to make Flink process configurable by env vars, e.g. 'some.yaml.option: val' -> FLINK_SOME_YAML_OPTION=val ## Extra files: jars, custom logging properties We can provide env vars to point to custom locations, e.g. in mounted volumes. # Extend image ## Python/hadoop versions, activating certain libs/plugins Users can install extra dependencies and change configs in their custom image which extends our base image. # Logging ## Web UI Modify the *log4j-console.properties* to also output logs into the files for WebUI. Limit log file size. ## Container output Separate effort for proper split of Flink process stdout and stderr into files and container output (idea with tee command: `program start-foreground &2>1 | tee flink-user-taskexecutor.out`) # Docker bash utils We are not going to expose it to users as an API. They should be able either to configure and run the standard entry point or the documentation should give short examples about how to extend and customise the base image. During the implementation, we will see if it makes sense to factor out certain bash procedures to reuse them e.g. in custom dev versions of docker image. # Dockerfile / image for developers We keep it on our future roadmap. This effort should help to understand what we can reuse there. Best, Andrey On Fri, Apr 3, 2020 at 12:57 PM Till Rohrmann <[hidden email]> wrote: > Hi everyone, > > just a small inline comment. > > On Fri, Apr 3, 2020 at 11:42 AM Ufuk Celebi <[hidden email]> wrote: > > > Hey Yang, > > > > thanks! See inline answers. > > > > On Fri, Apr 3, 2020 at 5:11 AM Yang Wang <[hidden email]> wrote: > > > > > Hi Ufuk, > > > > > > Thanks for make the conclusion and directly point out what need to be > > done > > > in > > > FLIP-111. I agree with you that we should narrow down the scope and > focus > > > the > > > most important and basic part about docker image unification. > > > > > > (1) Extend the entrypoint script in apache/flink-docker to start the > job > > >> cluster entry point > > > > > > I want to add a small requirement for the entry point script. > Currently, > > > for the native > > > K8s integration, we are using the apache/flink-docker image, but with > > > different entry > > > point("kubernetes-entry.sh"). Generate the java cmd in KubernetesUtils > > and > > > run it > > > in the entry point. I really hope it could merge to apache/flink-docker > > > "docker-entrypoint.sh". > > > > > > > The script [1] only adds the FLINK_CLASSPATH env var which seems > generally > > reasonable to me. But since principled classpath and entrypoint > > configuration is somewhat related to the follow-up improvement > proposals, I > > could also see this being done after FLIP-111. > > > > > > > (2) Extend the example log4j-console configuration > > >> => support log retrieval from the Flink UI out of the box > > > > > > If you mean to update the "flink-dist/conf/log4j-console.properties" to > > > support console and > > > local log files. I will say "+1". But we need to find a proper way to > > make > > > stdout/stderr output > > > both available for console and log files. Maybe till's proposal could > > help > > > to solve this. > > > "`program &2>1 | tee flink-user-taskexecutor.out`" > > > > > > > I think we can simply add a rolling file appender with a limit on the log > > size. > > > > I think this won't solve Yang's concern. What he wants to achieve is that > STDOUT and STDERR go to STDOUT and STDERR as well as into some *.out and > *.err file which are accessible from the web ui. I don't think that log > appender will help with this problem. > > Cheers, > Till > > > > – Ufuk > > > > [1] > > > > > https://github.com/apache/flink/blob/master/flink-dist/src/main/flink-bin/kubernetes-bin/kubernetes-entry.sh > > > |
Thanks for the summary, Andrey. Good idea to link Patrick's document from
the FLIP as a future direction so it doesn't get lost. Could you make sure to revive that discussion when FLIP-111 nears an end? This is good to go on my part. +1 to start the VOTE. @Till, @Yang: Thanks for the clarification with the output redirection. I didn't see that. The concern with the `tee` approach is that the file would grow indefinitely. I think we can solve this with regular logging by redirecting stderr to ERROR log level, but I'm not sure. We can look at a potential solution when we get to that point. :-) On Fri, Apr 3, 2020 at 3:36 PM Andrey Zagrebin <[hidden email]> wrote: > Hi everyone, > > Patrick and Ufuk, thanks a lot for more ideas and suggestions! > > I have updated the FLIP according to the current state of discussion. > Now it also contains the implementation steps and future follow-ups. > Please, review if there are any concerns. > The order of the steps aims for keeping Flink releasable at any point if > something does not have enough time to get in. > > It looks that we are reaching mostly a consensus for the open questions. > There is also a list of items, which have been discussed in this thread, > and short summary below. > As soon as there are no concerns, I will create a voting thread. > > I also added some thoughts for further customising logging setup. This may > be an optional follow-up > which is additional to the default logging into files for Web UI. > > # FLIP scope > The focus is users of the official releases. > Create docs for how to use the official docker image. > Remove other Dockerfiles in Flink repo. > Rely on running the official docker image in different modes (JM/TM). > Customise running the official image with env vars (This should minimise > manual manipulating of local files and creation of a custom image). > > # Base oficial image > > ## Java versions > There is a separate effort for this: > https://github.com/apache/flink-docker/pull/9 > > # Run image > > ## Entry point modes > JM session, JM job, TM > > ## Entry point config > We use env vars for this, e.g. FLINK_PROPERTIES and ENABLE_BUILT_IN_PLUGINS > > ## Flink config options > We document the existing FLINK_PROPERTIES env var to override config > options in flink-conf.yaml. > Then later, we do not need to expose and handle any other special env vars > for config options (address, port etc). > The future plan is to make Flink process configurable by env vars, e.g. > 'some.yaml.option: val' -> FLINK_SOME_YAML_OPTION=val > > ## Extra files: jars, custom logging properties > We can provide env vars to point to custom locations, e.g. in mounted > volumes. > > # Extend image > > ## Python/hadoop versions, activating certain libs/plugins > Users can install extra dependencies and change configs in their custom > image which extends our base image. > > # Logging > > ## Web UI > Modify the *log4j-console.properties* to also output logs into the files > for WebUI. Limit log file size. > > ## Container output > Separate effort for proper split of Flink process stdout and stderr into > files and container output > (idea with tee command: `program start-foreground &2>1 | tee > flink-user-taskexecutor.out`) > > # Docker bash utils > We are not going to expose it to users as an API. > They should be able either to configure and run the standard entry point > or the documentation should give short examples about how to extend and > customise the base image. > > During the implementation, we will see if it makes sense to factor out > certain bash procedures > to reuse them e.g. in custom dev versions of docker image. > > # Dockerfile / image for developers > We keep it on our future roadmap. This effort should help to understand > what we can reuse there. > > Best, > Andrey > > > On Fri, Apr 3, 2020 at 12:57 PM Till Rohrmann <[hidden email]> > wrote: > >> Hi everyone, >> >> just a small inline comment. >> >> On Fri, Apr 3, 2020 at 11:42 AM Ufuk Celebi <[hidden email]> wrote: >> >> > Hey Yang, >> > >> > thanks! See inline answers. >> > >> > On Fri, Apr 3, 2020 at 5:11 AM Yang Wang <[hidden email]> wrote: >> > >> > > Hi Ufuk, >> > > >> > > Thanks for make the conclusion and directly point out what need to be >> > done >> > > in >> > > FLIP-111. I agree with you that we should narrow down the scope and >> focus >> > > the >> > > most important and basic part about docker image unification. >> > > >> > > (1) Extend the entrypoint script in apache/flink-docker to start the >> job >> > >> cluster entry point >> > > >> > > I want to add a small requirement for the entry point script. >> Currently, >> > > for the native >> > > K8s integration, we are using the apache/flink-docker image, but with >> > > different entry >> > > point("kubernetes-entry.sh"). Generate the java cmd in KubernetesUtils >> > and >> > > run it >> > > in the entry point. I really hope it could merge to >> apache/flink-docker >> > > "docker-entrypoint.sh". >> > > >> > >> > The script [1] only adds the FLINK_CLASSPATH env var which seems >> generally >> > reasonable to me. But since principled classpath and entrypoint >> > configuration is somewhat related to the follow-up improvement >> proposals, I >> > could also see this being done after FLIP-111. >> > >> > >> > > (2) Extend the example log4j-console configuration >> > >> => support log retrieval from the Flink UI out of the box >> > > >> > > If you mean to update the "flink-dist/conf/log4j-console.properties" >> to >> > > support console and >> > > local log files. I will say "+1". But we need to find a proper way to >> > make >> > > stdout/stderr output >> > > both available for console and log files. Maybe till's proposal could >> > help >> > > to solve this. >> > > "`program &2>1 | tee flink-user-taskexecutor.out`" >> > > >> > >> > I think we can simply add a rolling file appender with a limit on the >> log >> > size. >> > >> > I think this won't solve Yang's concern. What he wants to achieve is >> that >> STDOUT and STDERR go to STDOUT and STDERR as well as into some *.out and >> *.err file which are accessible from the web ui. I don't think that log >> appender will help with this problem. >> >> Cheers, >> Till >> >> >> > – Ufuk >> > >> > [1] >> > >> > >> https://github.com/apache/flink/blob/master/flink-dist/src/main/flink-bin/kubernetes-bin/kubernetes-entry.sh >> > >> > |
Free forum by Nabble | Edit this page |