--> On Mon, Mar 16, 2020 at 1:58 AM Andrey Zagrebin <azagre...@apache.org> 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 <karma...@gmail.com> 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 <t...@apache.org> 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 <azagre...@apache.org > > > > 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 < > > konstan...@ververica.com> > > >> 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 <danrtsey...@gmail.com> > > 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 <azagre...@apache.org> 于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 > > >> > > > >