Skip to content

Conversation

zentol
Copy link
Contributor

@zentol zentol commented Oct 11, 2017

What is the purpose of the change

This PR integrates the LIST command into the RestClusterClient. This is a fully working implementation and leverages the CurrentJobsOverviewHeaders.

Brief change log

  • move LIST logic from CliFrontend into ClusterClient
  • separate CliFrontendListCancelTest into distinct test cases for list & cancel
  • implement LIST logic for RestClusterClient, based on the CurrentJobsOverviewHeaders

Verifying this change

This change added tests and can be verified as follows:

  • the changes to the CliFrontend are covered by modified tests in CliFrontendListTest
  • the changes to the ClusterClient are covered by new tests in ClusterClientTest
  • the changes to the RestClusterClient are covered by RestClusterClientTest#testListJobs

For manual verification:

  • start a flip6 cluster with start-cluster.sh flip6
  • submit a long-running job with flink run -flip <jar>
  • execute the list command with flink list -flip6

Does this pull request potentially affect one of the following parts:

  • Dependencies (does it add or upgrade a dependency): (no)
  • The public API, i.e., is any changed class annotated with @Public(Evolving): (no)
  • The serializers: (no)
  • The runtime per-record code paths (performance sensitive): (no)
  • Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper: (no)

Documentation

  • Does this pull request introduce a new feature? (no)
  • If yes, how is the feature documented? (not documented)

Copy link
Contributor

@tillrohrmann tillrohrmann left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM.

The only thing I was wondering about was whether the ClusterClient should return a Collection<JobStatus> instead of a CompletableFuture<Collection<JobStatus>>. I would assume that the user would want this operation to be executed synchronously, but I'm not entirely sure.

}
catch (Exception e) {
throw new Exception("Could not retrieve running jobs from the JobManager.", e);
CompletableFuture<Collection<JobDetails>> jobDetailsFuture = client.listJobs();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should we hide the fact that we are retrieving JobDetails internally and still only return a Collection<JobStatus>? The reason is that in the future we might change how we retrieve the job ids, job names plus their state.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Even if we change how we retrieve it we can still wrap them in a JobDetails object; it doesn't necessarily leak to the outside how we retrieve them. Given that the CLI requires the job id, job name, start/end timestamp and job status we aren't saving much by creating a slimmed down JobDetails class.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I thought about the JobStatusMessage class. I'm not entirely sure, because whatever you expose can be used and once we change that we no longer retrieve the full set of details for a job, it will be difficult to change it.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I can change it to that.

headers
);
return jobDetailsFuture
.thenApply(MultipleJobsDetails::getRunning);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we also have to return the list of completed jobs not only the running.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

we don't have to, as the LIST command only cares about running jobs, but we certainly could just return all. The CLI is filtering the jobs anyway.

import static org.powermock.api.mockito.PowerMockito.when;

/**
* Tests for the CANCEL command.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

typo: LIST command.

Collection<JobDetails> jobDetails = jobDetailsFuture.get();
Assert.assertTrue(gateway.messageArrived);
// finished jobs should be ignored
Assert.assertEquals(1, jobDetails.size());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Shouldn't we also retrieve the finished jobs here?

Copy link
Contributor Author

@zentol zentol Oct 11, 2017

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

No, the LIST command only lists running/scheduled jobs.

*
* @return future collection of running jobs
* @throws Exception if no connection to the cluster could be established
*/
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm wondering whether the ClusterClient should issue a synchronous operation instead of a asynchronous operation (simply waiting on the futures completion). So far all the ClusterClient operations are blocking and wait for the completion. Not entirely sure which is more intuitive to use for the user. Of course the asynchronous variant gives more freedom.

@zentol
Copy link
Contributor Author

zentol commented Oct 11, 2017

The reason why i went for the CompletableFuture route is to stay closer to the current code. The CliFrontend is waiting the the response and prints "Waiting...", which we can't do when doing the call synchronously since the ClusterClient shouldn't print anything.

That said, i do like making them asynchronous in general.

@zentol
Copy link
Contributor Author

zentol commented Oct 11, 2017

I've modified the clients to return both running and finished jobs.

@tillrohrmann
Copy link
Contributor

tillrohrmann commented Oct 11, 2017

I'm not entirely sure whether printing "Waiting..." is consistent with the status quo. The current list implementation does not do it. Moreover, so far all ClusterClient operations are synchronous (e.g. stop and cancel). In general I don't have strong feelings about returning a future or the result, but so far I think we always made the call synchronous and, thus, diverge from it with this change.

@zentol
Copy link
Contributor Author

zentol commented Oct 11, 2017

triggerSavepoint is also asynchronous. It is true that the list command does not print waiting, but the savepoint commands do.

@tillrohrmann
Copy link
Contributor

But triggerSavepoint is not a command of the ClusterClient but of the CliFrontend. All other ClusterClient commands like stop and cancel behave differently.

@zentol
Copy link
Contributor Author

zentol commented Oct 11, 2017

What do you mean when you say triggerSavepoint is not a command of the ClusterClient? It does have a triggerSavepoint method.

@tillrohrmann
Copy link
Contributor

True, you're right. I guess I overlooked it. But it also includes the trigger prefix which indicates that it does not take a savepoint but only triggers it. If we want to be consistent, then we would also return a future for the stop and cancel call on which you can wait if you want to wait for the completion of the call. But I guess that this part will be reworked anyway in the future when we refactor the client. Thus, I guess you can merge the PR.

@zentol
Copy link
Contributor Author

zentol commented Oct 15, 2017

merging.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants