Hi All! I fully acknowledge the general need to access more info about the running deployments. This need however is very specific to the use-cases / platforms built on the operator. I think we need a good way to tackle this without growing the status arbitrarily.
Currently the JobStatus in the operator contains the following fields: - jobId - state : Flink JobStatus - savepointInfo : Operator savepoint tracking info - startTime : Flink job startTime - updateTime : Last time state was updated in the operator - jobName: Name of the job Technically speaking only jobId, state and savepointInfo are used inside the operator logic, the rest is unnecessary and "could be removed" without affecting any operator functionality. I think instead of adding more of these "unnecessary/arbitrary" fields we should add a more generic way that allows a configurable / pluggable way to extend the status with user/platform specific fields based on the Flink job information. At the same time we should already @Deprecate / phase out the currently unnecessary fields. One way of doing this would be adding a new Map<String,String> metadata (or similar) field. And at the same time add a configurable / pluggable way to create the content of this metadata based on the Flink rest api response (the extended job details). What do you think? Gyula On Fri, Jul 15, 2022 at 1:05 PM WONG, DAREN <daren...@amazon.co.uk.invalid> wrote: > Hi Martin, > > Yes, that's understandable. I think adding job endTime, duration, jobPlan > is useful to other Flink users too as they now have info to track: > > 1. endTime: If the job has ended, the user can know when it has ended. If > the job is still streaming, then the user can know as it defaults to "-1". > 2. duration: Info on how long the job has been running for, useful for > monitoring purposes. > 3. jobPlan: Contains more detailed job info such as the operators in the > job graph and the parallelism of each operator. This could benefit Flink > users as follows: > 3.1. Help users to get a quick view on jobs simply by querying via > k8s API, without need to integrate with Flink Client/API. Useful for users > who mainly use kubectl. > 3.2. Allows users to easily notice a change in job. For eg, if > user changed a job code by adding a new operator but built it with same jar > name, then they can notice the change in jobPlan. > 3.3. User may want to operate on jobPlan difference. For eg, > create difference notification, allocate resources, or other automation > purposed. > > In general, I think adding these info is useful for Flink users from > simple monitoring to audit trail purposes. In addition, these info are > available via Flink REST API, hence I believe Flink users who tracks these > info via API would benefit from them when they start using Flink Kubernetes > Operator. > > Regards, > Daren > > > On 13/07/2022, 08:25, "Martijn Visser" <martijnvis...@apache.org> wrote: > > CAUTION: This email originated from outside of the organization. Do > not click links or open attachments unless you can confirm the sender and > know the content is safe. > > > > Hi Daren, > > Could you list the benefits for the users of Flink? I do think that an > internal AWS requirement is not a good argument for getting something > done > in Flink. > > Best regards, > > Martijn > > Op di 12 jul. 2022 om 21:17 schreef WONG, DAREN > <daren...@amazon.co.uk.invalid>: > > > Hi Yang, > > > > The requirement to add *plan* currently originates from an internal > AWS > > requirement as our service needs visibility of *plan*, but we think > it > > could be beneficial as well to customers who uses *plan* too. > > > > Regards, > > Daren > > > > > > > > > > On 12/07/2022, 13:23, "Yang Wang" <danrtsey...@gmail.com> wrote: > > > > CAUTION: This email originated from outside of the organization. > Do > > not click links or open attachments unless you can confirm the > sender and > > know the content is safe. > > > > > > > > Thanks for the explanation. Only having 1 API call in most cases > makes > > sense to me. > > > > Could you please elaborate more about why do we need the *plan* > in CR > > status? > > > > > > Best, > > Yang > > > > Gyula Fóra <gyula.f...@gmail.com> 于2022年7月12日周二 17:36写道: > > > > > Hi Devs! > > > > > > I discussed with Daren offline, and I agree with him that > > technically we > > > almost never need 2 API calls. > > > > > > I think it's fine to have a second API call once directly after > > application > > > submission (technically even this can be eliminated by setting > a fix > > job id > > > always). > > > > > > +1 from me. > > > > > > Cheers, > > > Gyula > > > > > > > > > On Tue, Jul 12, 2022 at 11:32 AM WONG, DAREN > > <daren...@amazon.co.uk.invalid > > > > > > > wrote: > > > > > > > Hi Matyas, > > > > > > > > Thanks for the feedback, and yes I agree. An alternative > approach > > would > > > > instead be: > > > > > > > > - 2 API calls only when jobID is not available (i.e when > > submitting a new > > > > application cluster, which is a one-off event). > > > > - 1 API call when jobID is already available by directly > calling > > > > "/jobs/:jobid". > > > > > > > > With this approach, we can keep the API call to 1 in most > cases. > > > > > > > > Regards, > > > > Daren > > > > > > > > > > > > On 11/07/2022, 14:44, "Őrhidi Mátyás" < > matyas.orh...@gmail.com> > > wrote: > > > > > > > > CAUTION: This email originated from outside of the > > organization. Do > > > > not click links or open attachments unless you can confirm > the > > sender and > > > > know the content is safe. > > > > > > > > > > > > > > > > Hi Daren, > > > > > > > > At the moment the Operator fetches the job state via > > > > > > > > > > > > > > https://nightlies.apache.org/flink/flink-docs-master/docs/ops/rest_api/#jobs-overview > > > > which contains the 'end-time' and 'duration' fields > already. I > > feel > > > > calling > > > > the > > > > > > > > > > > > > > https://nightlies.apache.org/flink/flink-docs-master/docs/ops/rest_api/#jobs-jobid > > > > after the previous call for every job in every reconcile > loop > > would > > > be > > > > too > > > > expensive. > > > > > > > > Best, > > > > Matyas > > > > > > > > On Mon, Jul 11, 2022 at 3:17 PM WONG, DAREN > > > > <daren...@amazon.co.uk.invalid> > > > > wrote: > > > > > > > > > Hi everyone, I am Daren from AWS Kinesis Data Analytics > > (KDA) team. > > > > I had > > > > > a quick chat with Gyula as I propose to include a few > > additional > > > > fields in > > > > > the jobStatus CRD for Flink Kubernetes Operator such > as: > > > > > > > > > > - endTime > > > > > - duration > > > > > - jobPlan > > > > > > > > > > Further details of each states can be found here< > > > > > > > > > > > > > > > https://github.com/darenwkt/flink/blob/release-1.15.0/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/job/JobDetailsInfo.java > > > > >. > > > > > Although addition of these 3 states stem from an > internal > > > > requirement, I > > > > > think they would be beneficial to others who uses these > > states in > > > > their > > > > > application as well. The list of states above are not > > exhaustive, > > > so > > > > do let > > > > > me know if there are other states that you would like > to > > include > > > > together > > > > > in this iteration cycle. > > > > > > > > > > JIRA: > https://issues.apache.org/jira/browse/FLINK-28494 > > > > > > > > > > > > > > > > > > > > > >