Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[FLINK-7647] [flip6] Port JobManagerConfigHandler to new REST endpoint #4691

Closed
wants to merge 4 commits into from

Conversation

tzulitai
Copy link
Contributor

What is the purpose of the change

This PR ports the existing JobManagerConfigHandler to the new REST endpoint. This includes introducing the ClusterConfiguration response and ClusterConfigurationHeaders. The DispatcherRestEndpoint now registers the JobManagerConfigHandler.

Additionally, this PR also contains other cosmetic changes, such as properly renaming the JobManagerConfigHandler to ClusterConfigHandler, and refactoring common test logic for marshalling / unmarshalling of REST responses.

Brief change log

  • Let JobManagerConfigHandler implement LegacyRestEHandler
  • Register JobManagerConfigHandler at DispatcherRestEndpoint
  • Rename JobManagerConfigHandler to ClusterConfigHandler
  • Introduce RestResponseMarshallingTestBase

Verifying this change

This change is a trivial rework / code cleanup.
Only additional test is ClusterConfigurationTest for the marshalling of the ClusterConfiguration.

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: yes

Documentation

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

@tillrohrmann
Copy link
Contributor

Thanks for your contribution @tzulitai. There are still some checkstyle violations in the code.

This commit lets the JobManagerConfigHandler implement the
LegacyRestHandler interface in order to be ported to the new REST
endpoint. This includes the introduction of ClusterConfiguration
response body and ClusterConfigurationHeaders.

The DispatcherRestEndpoint now also registers the
JobManagerConfigHandler.
@tzulitai
Copy link
Contributor Author

@tillrohrmann I've corrected checkstyles violations and rebased.

…andler

The original naming wouldn't make sense for the FLIP-6 redesign, since
we would have multiple per-job JobManagers for each cluster, which
shares the same configuration.

The REST path is still left untouched and not part of this commit, as
that would involve more changes in flink-runtime-web.
Introduces a common test base that for all REST responses, a subclass
should be implemented to verify that the response can be correctly
marshalled and unmarshalled.
@tillrohrmann
Copy link
Contributor

Still failing with ClusterConfigHandlerTest.testGetPaths:32 NullPointer

@tzulitai
Copy link
Contributor Author

Sorry about that, local Travis tests pass, should be fine now.

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.

Changes look good to me @tzulitai. I had only minor comments. After addressing them, we can merge this PR :-)

private final RestHandlerConfiguration restConfiguration;
private final Executor executor;

public DispatcherRestEndpoint(
RestServerEndpointConfiguration configuration,
Configuration clusterConfiguration,
RestServerEndpointConfiguration endpointConfiguration,
Copy link
Contributor

Choose a reason for hiding this comment

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

Shall we remove the order between clusterConfiguration and endpointConfiguration? I usually like to have the super class arguments first.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Will change.

HandlerRequest<EmptyRequestBody, EmptyMessageParameters> request,
DispatcherGateway gateway) {

return CompletableFuture.supplyAsync(() -> ClusterConfiguration.from(config), executor);
Copy link
Contributor

Choose a reason for hiding this comment

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

Maybe we could generate the ClusterConfiguration instance once at creation time and then always return this element as a CompletableFuture.completedFuture(clusterConfiguration).

Copy link
Contributor Author

Choose a reason for hiding this comment

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

That makes a lot of sense, yes. Will change.

* Response of the {@link ClusterConfigHandler}, respresented as a list
* of key-value pairs of the cluster {@link Configuration}.
*/
public class ClusterConfiguration extends ArrayList<ClusterConfigurationEntry> implements ResponseBody {
Copy link
Contributor

Choose a reason for hiding this comment

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

Serial version uid missing.

* Response of the {@link ClusterConfigHandler}, respresented as a list
* of key-value pairs of the cluster {@link Configuration}.
*/
public class ClusterConfiguration extends ArrayList<ClusterConfigurationEntry> implements ResponseBody {
Copy link
Contributor

Choose a reason for hiding this comment

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

There is already another class called ClusterConfiguration. Maybe we should rename it in order to disambiguate 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'll rename this to ClusterConfigurationInfo.

* Response of the {@link ClusterConfigHandler}, respresented as a list
* of key-value pairs of the cluster {@link Configuration}.
*/
public class ClusterConfiguration extends ArrayList<ClusterConfigurationEntry> implements ResponseBody {
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 actually wondering why we don't extend from a Map implementation since it should behave more like a map. This of course would require changes on the web gui side. I think now, the JSON would like

[{"key":"keyvalue", "value":"valuevalue"}, {....}] 

instead of

{"keyvalue":"valuevalue", ....}

Maybe we could change this in the future.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes, I found the original use of the JSONArray structure very strange also.
We can include a change for this probably when we also re-work the REST resource path for this handler.

@tzulitai
Copy link
Contributor Author

Thanks a lot for the review @tillrohrmann. I'll go ahead and merge this PR after addressing your comments.

tzulitai added a commit to tzulitai/flink that referenced this pull request Sep 26, 2017
Introduces a common test base that for all REST responses, a subclass
should be implemented to verify that the response can be correctly
marshalled and unmarshalled.

This closes apache#4691.
This closes apache#4720.
tzulitai added a commit to tzulitai/flink that referenced this pull request Sep 26, 2017
Introduces a common test base that for all REST responses, a subclass
should be implemented to verify that the response can be correctly
marshalled and unmarshalled.

This closes apache#4691.
This closes apache#4720.
tzulitai added a commit to tzulitai/flink that referenced this pull request Sep 26, 2017
Introduces a common test base that for all REST responses, a subclass
should be implemented to verify that the response can be correctly
marshalled and unmarshalled.

This closes apache#4691.
This closes apache#4720.
@asfgit asfgit closed this in 4ba3eec Sep 26, 2017
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
3 participants