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-7709] Add CheckpointStatisticDetailsHandler for new REST endpoint #4763

Conversation

tillrohrmann
Copy link
Contributor

What is the purpose of the change

Adds the new CheckpointStatisticDetailsHandler for the new REST server endpoint.

Moreover, this PR disables the FAIL_ON_MISSING_CREATOR_PROPERTIES property for the RestMapperUtils.getStrictObjectMapper because that is something the individuals beans can do on their own (e.g. by checking with Preconditions.checkNotNull).

R @zentol because of the changes to the ObjectMapper setup.

Verifying this change

This change is already covered by existing tests, such as CheckpointingStatisticsTest.

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)

@tillrohrmann
Copy link
Contributor Author

Rebasing onto the latest master.

@tillrohrmann tillrohrmann force-pushed the portCheckpointStatsDetailsHandler branch from 4d69441 to d152cf2 Compare October 4, 2017 13:05
@zentol
Copy link
Contributor

zentol commented Oct 4, 2017

Reviewed 24 of 31 files at r1.
Review status: 21 of 26 files reviewed at latest revision, 2 unresolved discussions, some commit checks failed.


flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/checkpoints/CheckpointStatsCache.java, line 44 at r1 (raw file):

	private final Cache<Long, AbstractCheckpointStats> cache;

	public CheckpointStatsCache(int maxNumEntries) {

The CheckpointStatsCache should be moved out of the legacy namespace.


flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/checkpoints/CheckpointStatistics.java, line 233 at r1 (raw file):

				}
			} else {
				checkpointStatisticsPerTask = Collections.emptyMap();

This method is near identical to CheckpointingStatistics#generate..., except this one line.


Comments from Reviewable

@tillrohrmann
Copy link
Contributor Author

@zentol I'm not entirely sure whether reviewable makes things better or worse wrt reviewing. At least I can no longer respond to individual comments directly from github.

You're right with both comments. I'll remove the CheckpointingStatistics#generate method. Moreover, I'll move the CheckpointStatsCache out of the legacy package.

@zentol
Copy link
Contributor

zentol commented Oct 5, 2017

@tillrohrmann I wanted to try it out, primarily since i can mark individual files as reviewed. For the remaining files I will once again write the comments on github.

@tillrohrmann
Copy link
Contributor Author

Alright @zentol. I guess it would work if I signed up for reviewable.

@tillrohrmann tillrohrmann force-pushed the portCheckpointStatsDetailsHandler branch from 44dfe88 to 41434f7 Compare October 5, 2017 12:50
@tillrohrmann
Copy link
Contributor Author

Quick question @zentol. You've reviewed and approved #4772 which is based on this one. Do you have more comments for this PR?

@zentol
Copy link
Contributor

zentol commented Oct 9, 2017

I still have to look at the JSON generation in this PR.

@JsonProperty(FIELD_NAME_NUM_SUBTASKS) int numSubtasks,
@JsonProperty(FIELD_NAME_NUM_ACK_SUBTASKS) int numAckSubtasks,
@JsonDeserialize(keyUsing = JobVertexIDDeserializer.class) @JsonProperty(FIELD_NAME_TASKS) @Nullable Map<JobVertexID, TaskCheckpointStatistics> checkpointStatisticsPerTask) {
this.id = id;
Copy link
Contributor

Choose a reason for hiding this comment

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

missing checkNotNull check for all fields but checkpoitnStatisticsPerTask.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Good catch. Will add the checks.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Hmm I think all not checked parameters are actually primitives and, thus, don't need to be checked for null.

Copy link
Contributor

Choose a reason for hiding this comment

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

true, all objects are checked.

@JsonSubTypes({
@JsonSubTypes.Type(value = CheckpointStatistics.CompletedCheckpointStatistics.class, name = "completed"),
@JsonSubTypes.Type(value = CheckpointStatistics.FailedCheckpointStatistics.class, name = "failed")})
@JsonInclude(JsonInclude.Include.NON_NULL)
Copy link
Contributor

Choose a reason for hiding this comment

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

We could annotate the particular field instead of the class.

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 will remove the Include.NON_NULL directive and re-add the FAIL_ON_MISSING_CREATOR_PROPERTY to the ObjectMapper.

throw new IllegalArgumentException("Given checkpoint stats object of type " + checkpointStats.getClass().getName() + " cannot be converted.");
}
} else {
return null;
Copy link
Contributor

Choose a reason for hiding this comment

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

pretty sure this would lead to a NullPointerException, since neither the CheckpointStatshandler, not AbstractExceutionGraphhandler, nor the AbstractRestHandler handle the case of the returned value being null.

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 think it should actually work and output a serialized null value. However, this case distinction is not necessary and should be better pulled out of this method.

numAckSubtasks,
checkpointingStatisticsPerTask);

this.externalPath = externalPath;
Copy link
Contributor

Choose a reason for hiding this comment

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

missing null checks

numAckSubtasks,
checkpointingStatisticsPerTask);

this.failureTimestamp = failureTimestamp;
Copy link
Contributor

Choose a reason for hiding this comment

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

missing null checks

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Should be a primitive.

@@ -33,8 +33,7 @@
objectMapper.enable(
DeserializationFeature.FAIL_ON_IGNORED_PROPERTIES,
DeserializationFeature.FAIL_ON_NULL_FOR_PRIMITIVES,
DeserializationFeature.FAIL_ON_READING_DUP_TREE_KEY,
DeserializationFeature.FAIL_ON_MISSING_CREATOR_PROPERTIES);
Copy link
Contributor

Choose a reason for hiding this comment

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

Removing this also means we have to add explicit null checks to all existing Request-/ResponseBody classes.

Copy link
Contributor

Choose a reason for hiding this comment

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

Do note that we must be on the look-out for requests that use primitive fields, as jackson will default them to 0 if they are missing, which will cause misleading error messages.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This is a valid objection which I share. I'll remove this change and set the map of TaskCheckpointStatistics to an empty map in case that we want to leave the details out.

@tillrohrmann
Copy link
Contributor Author

Thanks for the review @zentol. I've addressed your comments: Reverted change to RestMapperUtils, removed null branch from CheckpointStatistics#generateCheckpointStatistics; Made the CheckpointStatistics#checkpointStatisticsPerTask non nullable. If you have no further objections, then I would like to merge this PR once Travis gives green light.

…ointingStatistics#generateCheckpointStatistics method
…ull check out of CheckpointStatistics#generateCheckpointStatistics; Make CheckpointStatistics#checkpointStatisticcsPerTask non nullable; Add fail on missing creator property
@tillrohrmann tillrohrmann force-pushed the portCheckpointStatsDetailsHandler branch from a15f452 to 147a5b0 Compare October 10, 2017 13:18
@tillrohrmann
Copy link
Contributor Author

Rebased onto the latest master

@zentol
Copy link
Contributor

zentol commented Oct 10, 2017

no objections, feel free to merge this.

tillrohrmann added a commit to tillrohrmann/flink that referenced this pull request Oct 10, 2017
Disable failing when not all creator properties are known

Move CheckpointStatsCache out of legacy package; Remove unused CheckpointingStatistics#generateCheckpointStatistics method

Remove JsonInclude.Include.NON_NULL from CheckpointStatistics; Pull null check out of CheckpointStatistics#generateCheckpointStatistics; Make CheckpointStatistics#checkpointStatisticcsPerTask non nullable; Add fail on missing creator property

This closes apache#4763.
@tillrohrmann tillrohrmann deleted the portCheckpointStatsDetailsHandler branch October 10, 2017 15:36
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
3 participants