You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by tillrohrmann <gi...@git.apache.org> on 2017/08/26 20:41:36 UTC

[GitHub] flink pull request #4604: [FLINK-7535] Port DashboardConfigHandler to new RE...

GitHub user tillrohrmann opened a pull request:

    https://github.com/apache/flink/pull/4604

    [FLINK-7535] Port DashboardConfigHandler to new REST endpoint

    ## What is the purpose of the change
    
    Lets DashboardConfigHandler implement the LegacyRestHandler. Moreover, this
    commit defines the appropriate DashboardConfigurationHeaders.
    
    The DispatcherRestEndpoint registers the DashboardConfigHandler.
    
    ## Verifying this change
    
    This change is a trivial rework / code cleanup without any test coverage.
    
    ## 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 applicable)
    


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/tillrohrmann/flink portDashboardConfigHandler

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/4604.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #4604
    
----
commit 7835d0a89c8ecdd5b9661ee8c57a9d63a3ed3742
Author: Till Rohrmann <tr...@apache.org>
Date:   2017-08-14T13:47:04Z

    [FLINK-7522] Add termination future to ClusterEntrypoint
    
    The termination future is completed when the ClusterEntrypoint shuts down. This
    allows for easier testing.

commit 2cdf97f824bc62a82e65f4c160b9ad64de446de4
Author: Till Rohrmann <tr...@apache.org>
Date:   2017-08-16T12:36:13Z

    [FLINK-7457] Make Dispatcher highly available
    
    This commit introduces a dispatcher leader election and retrieval service to the
    HighAvailabilityServices. Moreover it adds code such that the Dispatcher now takes
    part in the leader election process using the afore-mentioned services.
    
    Let Dispatcher participate in leader election
    
    Add test for Dispatcher leader election

commit 04caf85d33ddfc3a4a9b788745b8282c3437d8e2
Author: Till Rohrmann <tr...@apache.org>
Date:   2017-08-10T08:56:12Z

    [FLINK-7409] [web] Make WebRuntimeMonitor reactive
    
    This commit changes the behaviour of the WebRuntimeMonitor to not longer block serving
    threads by waiting on the result of futures. Instead the RequestHandler now returns a
    CompletableFuture<FullHttpResponse> which is written out to the Netty channel upon
    completion. This will improve the performance of our WebRuntimeMonitor.

commit 4fa6dedd95555a2d1a91339ff5effda3bc2bd1d5
Author: Till Rohrmann <tr...@apache.org>
Date:   2017-08-15T10:00:58Z

    [FLINK-7458] Generalize GatewayRetriever for WebRuntimeMonitor
    
    Introduce a generalized GatewayRetriever replacing the JobManagerRetriever. The
    GatewayRetriever fulfills the same purpose as the JobManagerRetriever with the
    ability to retrieve the gateway for an arbitrary endpoint type.

commit 0f9b2ce77e20f25fc95ddeba98f863b86450a72c
Author: Till Rohrmann <tr...@apache.org>
Date:   2017-08-15T11:55:47Z

    [FLINK-7459] Generalize Flink's redirection logic
    
    Introduce RedirectHandler which can be extended to add redirection functionality to all
    SimpleInboundChannelHandlers. This allows to share the same functionality across the
    StaticFileServerHandler and the RuntimeMonitorHandlerBase which could now be removed.
    In the future, the AbstractRestHandler will also extend the RedirectHandler.

commit 88aed4f7a198b3994271088b8e19558d399ddd9d
Author: Till Rohrmann <tr...@apache.org>
Date:   2017-08-17T13:04:19Z

    [FLINK-7527] [rest] Let AbstractRestHandler extend RedirectHandler
    
    By letting the AbstractRestHandler extend the RedirectHandler, we add redirection
    capabilities to the AbstractRestHandler.

commit 9b7de1dc21b771fc10ee1661f34c142e990b424f
Author: Till Rohrmann <tr...@apache.org>
Date:   2017-08-17T13:25:33Z

    [FLINK-7528] Create DispatcherRestEndpoint and integrate with Dispatcher
    
    This commit creates the DispatcherRestEndpoint and integrates it with the
    Dispatcher. The DispatcherRestEndpoint is created in the SessionClusterEntrypoint
    and its address is passed to the Dispatcher such that it can answer the
    requestRestAddress RPC.

commit 5e1d0962e654038894fa376e86b1eeb1565c851b
Author: Till Rohrmann <tr...@apache.org>
Date:   2017-08-17T14:55:03Z

    [FLINK-7529] Retrieve complete REST address from gateway
    
    With this change, the complete REST address (protocol://hostname:port) is retrieved
    from the RestfulGateway. That way we decouple the RestHandlers from the underlying
    RestServerEndpoint/WebRuntimeMonitor because they no longer have to know whether
    HTTPs is enabled or not.

commit 7de9e4fa387bf355b0b3d5b520f522b80b3f14fa
Author: Till Rohrmann <tr...@apache.org>
Date:   2017-08-18T07:52:30Z

    [FLINK-7531] Move Flink legacy rest handler to flink-runtime
    
    Move metrics handlers under o.a.f.runtime.webmonitor.handlers
    
    Move StaticFileServerHandler under o.a.f.runtime.webmonitor.files

commit cffb1ce128a5f6c5f2a3a9461ce1895fe0269273
Author: Till Rohrmann <tr...@apache.org>
Date:   2017-08-18T12:05:11Z

    [FLINK-7532] Add web content handler to DispatcherRestEndpoint
    
    Adds the StaticFileContentHandler to the DispatcherRestEndpoint if the
    flink-runtime-web dependency is in the classpath. In order to setup the
    respective channel handler, this commit introduces the setupChannelHandlers
    method to the RestServerEndpoint.

commit e8ef04c261905d551eb270fd22895363d34c22f3
Author: Till Rohrmann <tr...@apache.org>
Date:   2017-08-18T12:29:29Z

    [FLINK-7533] Let LeaderGatewayRetriever retry failed gateway retrievals
    
    Add test case

commit e809638dd76c9043c4215704b74455a9177cf1be
Author: Till Rohrmann <tr...@apache.org>
Date:   2017-08-18T14:18:19Z

    [FLINK-7534] Create LegacyRestHandlerAdapter for old REST handlers
    
    Introduce LegacyRestHandler interface which the old REST handler have to implement
    in order to make them usable for the RestServerEndpoint in combination with the
    LegacyRestHandlerAdapter. The LegacyRestHandlerAdapter extends the AbstractRestHandler
    and runs the LegacyRestHandler implementation.
    
    As an example, this commit ports the ClusterOverviewHandler to the new interface. The
    Dispatcher side still has to be properly implemented.

commit cfc571b757b40e55fd7778a345fc79fdc51a6521
Author: Till Rohrmann <tr...@apache.org>
Date:   2017-08-21T13:11:08Z

    [FLINK-7535] Port DashboardConfigHandler to new REST endpoint
    
    Lets DashboardConfigHandler implement the LegacyRestHandler. Moreover, this
    commit defines the appropriate DashboardConfigurationHeaders.
    
    The DispatcherRestEndpoint registers the DashboardConfigHandler.

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #4604: [FLINK-7535] Port DashboardConfigHandler to new REST endp...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on the issue:

    https://github.com/apache/flink/pull/4604
  
    Thanks for the review @zentol and @tzulitai. Merging this PR now.


---

[GitHub] flink pull request #4604: [FLINK-7535] Port DashboardConfigHandler to new RE...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4604#discussion_r139916255
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/DashboardConfigurationHeaders.java ---
    @@ -0,0 +1,60 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.runtime.rest.messages;
    +
    +import org.apache.flink.runtime.rest.HttpMethodWrapper;
    +import org.apache.flink.runtime.rest.handler.legacy.DashboardConfigHandler;
    +import org.apache.flink.runtime.rest.handler.legacy.messages.DashboardConfiguration;
    +
    +import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
    +
    +/**
    + * Message headers for the {@link DashboardConfigHandler}.
    + */
    +public class DashboardConfigurationHeaders implements MessageHeaders<EmptyRequestBody, DashboardConfiguration, EmptyMessageParameters> {
    --- End diff --
    
    True. The same applies to `ClusterOverviewHeaders`. Will change it.


---

[GitHub] flink pull request #4604: [FLINK-7535] Port DashboardConfigHandler to new RE...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4604#discussion_r139905717
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/messages/DashboardConfiguration.java ---
    @@ -0,0 +1,112 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.runtime.rest.handler.legacy.messages;
    +
    +import org.apache.flink.runtime.rest.handler.legacy.DashboardConfigHandler;
    +import org.apache.flink.runtime.rest.messages.ResponseBody;
    +import org.apache.flink.runtime.util.EnvironmentInformation;
    +import org.apache.flink.util.Preconditions;
    +
    +import com.fasterxml.jackson.annotation.JsonProperty;
    +
    +import java.time.ZonedDateTime;
    +import java.time.format.TextStyle;
    +import java.util.Locale;
    +
    +/**
    + * Response of the {@link DashboardConfigHandler} containing general configuration
    + * values such as the time zone and the refresh interval.
    + */
    +public class DashboardConfiguration implements ResponseBody {
    +
    +	public static final String FIELD_NAME_REFRESH_INTERVAL = "refresh-interval";
    +	public static final String FIELD_NAME_TIMEZONE_OFFSET = "timezone-offset";
    +	public static final String FIELD_NAME_TIMEZONE_NAME = "timezone-name";
    +	public static final String FIELD_NAME_FLINK_VERSION = "flink-version";
    +	public static final String FIELD_NAME_FLINK_REVISION = "flink-revision";
    +
    +	@JsonProperty(FIELD_NAME_REFRESH_INTERVAL)
    +	private final long refreshInterval;
    +
    +	@JsonProperty(FIELD_NAME_TIMEZONE_NAME)
    +	private final String timeZoneName;
    +
    +	@JsonProperty(FIELD_NAME_TIMEZONE_OFFSET)
    +	private final int timeZoneOffset;
    +
    +	@JsonProperty(FIELD_NAME_FLINK_VERSION)
    +	private final String flinkVersion;
    +
    +	private final String flinkRevision;
    --- End diff --
    
    this field is missing the Jackson annotation.


---

[GitHub] flink pull request #4604: [FLINK-7535] Port DashboardConfigHandler to new RE...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4604#discussion_r139907889
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/RestHandlerConfiguration.java ---
    @@ -0,0 +1,66 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.runtime.rest.handler;
    +
    +import org.apache.flink.api.common.time.Time;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.configuration.WebOptions;
    +import org.apache.flink.util.Preconditions;
    +
    +import java.io.File;
    +
    +/**
    + * Configuration object containing values for the rest handler configuration.
    + */
    +public class RestHandlerConfiguration {
    +
    +	private final long refreshInterval;
    +
    +	private final Time timeout;
    +
    +	private final File tmpDir;
    +
    +	public RestHandlerConfiguration(long refreshInterval, Time timeout, File tmpDir) {
    +		this.refreshInterval = refreshInterval;
    --- End diff --
    
    Is it required that we have a validation check for this? Although I don't think we ever had a validation check for this before.


---

[GitHub] flink pull request #4604: [FLINK-7535] Port DashboardConfigHandler to new RE...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4604#discussion_r139915435
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/DashboardConfigurationHeaders.java ---
    @@ -0,0 +1,60 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.runtime.rest.messages;
    +
    +import org.apache.flink.runtime.rest.HttpMethodWrapper;
    +import org.apache.flink.runtime.rest.handler.legacy.DashboardConfigHandler;
    +import org.apache.flink.runtime.rest.handler.legacy.messages.DashboardConfiguration;
    +
    +import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
    +
    +/**
    + * Message headers for the {@link DashboardConfigHandler}.
    + */
    +public class DashboardConfigurationHeaders implements MessageHeaders<EmptyRequestBody, DashboardConfiguration, EmptyMessageParameters> {
    --- End diff --
    
    This could be a singleton


---

[GitHub] flink pull request #4604: [FLINK-7535] Port DashboardConfigHandler to new RE...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4604#discussion_r139908899
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/messages/DashboardConfiguration.java ---
    @@ -0,0 +1,112 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.runtime.rest.handler.legacy.messages;
    +
    +import org.apache.flink.runtime.rest.handler.legacy.DashboardConfigHandler;
    +import org.apache.flink.runtime.rest.messages.ResponseBody;
    +import org.apache.flink.runtime.util.EnvironmentInformation;
    +import org.apache.flink.util.Preconditions;
    +
    +import com.fasterxml.jackson.annotation.JsonProperty;
    +
    +import java.time.ZonedDateTime;
    +import java.time.format.TextStyle;
    +import java.util.Locale;
    +
    +/**
    + * Response of the {@link DashboardConfigHandler} containing general configuration
    + * values such as the time zone and the refresh interval.
    + */
    +public class DashboardConfiguration implements ResponseBody {
    +
    +	public static final String FIELD_NAME_REFRESH_INTERVAL = "refresh-interval";
    +	public static final String FIELD_NAME_TIMEZONE_OFFSET = "timezone-offset";
    +	public static final String FIELD_NAME_TIMEZONE_NAME = "timezone-name";
    +	public static final String FIELD_NAME_FLINK_VERSION = "flink-version";
    +	public static final String FIELD_NAME_FLINK_REVISION = "flink-revision";
    +
    +	@JsonProperty(FIELD_NAME_REFRESH_INTERVAL)
    +	private final long refreshInterval;
    +
    +	@JsonProperty(FIELD_NAME_TIMEZONE_NAME)
    +	private final String timeZoneName;
    +
    +	@JsonProperty(FIELD_NAME_TIMEZONE_OFFSET)
    +	private final int timeZoneOffset;
    +
    +	@JsonProperty(FIELD_NAME_FLINK_VERSION)
    +	private final String flinkVersion;
    +
    +	private final String flinkRevision;
    --- End diff --
    
    Good catch. I'll add the marshalling test.


---

[GitHub] flink pull request #4604: [FLINK-7535] Port DashboardConfigHandler to new RE...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4604#discussion_r139908984
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/RestHandlerConfiguration.java ---
    @@ -0,0 +1,66 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.runtime.rest.handler;
    +
    +import org.apache.flink.api.common.time.Time;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.configuration.WebOptions;
    +import org.apache.flink.util.Preconditions;
    +
    +import java.io.File;
    +
    +/**
    + * Configuration object containing values for the rest handler configuration.
    + */
    +public class RestHandlerConfiguration {
    +
    +	private final long refreshInterval;
    +
    +	private final Time timeout;
    +
    +	private final File tmpDir;
    +
    +	public RestHandlerConfiguration(long refreshInterval, Time timeout, File tmpDir) {
    +		this.refreshInterval = refreshInterval;
    --- End diff --
    
    Adding a sanity check wouldn't hurt. I'll add it.


---

[GitHub] flink pull request #4604: [FLINK-7535] Port DashboardConfigHandler to new RE...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/flink/pull/4604


---

[GitHub] flink pull request #4604: [FLINK-7535] Port DashboardConfigHandler to new RE...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4604#discussion_r139907431
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/messages/DashboardConfiguration.java ---
    @@ -0,0 +1,112 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.runtime.rest.handler.legacy.messages;
    +
    +import org.apache.flink.runtime.rest.handler.legacy.DashboardConfigHandler;
    +import org.apache.flink.runtime.rest.messages.ResponseBody;
    +import org.apache.flink.runtime.util.EnvironmentInformation;
    +import org.apache.flink.util.Preconditions;
    +
    +import com.fasterxml.jackson.annotation.JsonProperty;
    +
    +import java.time.ZonedDateTime;
    +import java.time.format.TextStyle;
    +import java.util.Locale;
    +
    +/**
    + * Response of the {@link DashboardConfigHandler} containing general configuration
    + * values such as the time zone and the refresh interval.
    + */
    +public class DashboardConfiguration implements ResponseBody {
    +
    +	public static final String FIELD_NAME_REFRESH_INTERVAL = "refresh-interval";
    +	public static final String FIELD_NAME_TIMEZONE_OFFSET = "timezone-offset";
    +	public static final String FIELD_NAME_TIMEZONE_NAME = "timezone-name";
    +	public static final String FIELD_NAME_FLINK_VERSION = "flink-version";
    +	public static final String FIELD_NAME_FLINK_REVISION = "flink-revision";
    +
    +	@JsonProperty(FIELD_NAME_REFRESH_INTERVAL)
    +	private final long refreshInterval;
    +
    +	@JsonProperty(FIELD_NAME_TIMEZONE_NAME)
    +	private final String timeZoneName;
    +
    +	@JsonProperty(FIELD_NAME_TIMEZONE_OFFSET)
    +	private final int timeZoneOffset;
    +
    +	@JsonProperty(FIELD_NAME_FLINK_VERSION)
    +	private final String flinkVersion;
    +
    +	private final String flinkRevision;
    --- End diff --
    
    Which means we would want a Json marshalling test for this also.


---