KAFKA-6376: streams skip metrics#4812
KAFKA-6376: streams skip metrics#4812vvcephei wants to merge 4 commits intoapache:trunkfrom vvcephei:kip-274-streams-skip-metrics
Conversation
|
Just starting on this; still need to add the missing measurements and actually run the tests. |
guozhangwang
left a comment
There was a problem hiding this comment.
I have not seen the skippedRecordsSensor to be recorded for other reasons yet, but I think from processorContext it should be able to include the record metadata on all such occurrences.
There was a problem hiding this comment.
I think you can still use the log4j format here, e.g.
"Skipping record due to deserialization error. topic={} partition={} offset={}", rawRecord.topic(), rawRecord.partition(), rawRecord.offset(), deserializationException
With four parameters, the last one is auto interpreted as the exception; maybe we can validate if this is the case.
There was a problem hiding this comment.
Confirmed, switching to the variant you mentioned still prints:
[2018-04-04 18:23:20,310] WARN task [0_1] Skipping record due to deserialization error. topic=[topic1] partition=[1] offset=[1] (org.apache.kafka.streams.processor.internals.RecordDeserializer:80)
org.apache.kafka.common.errors.SerializationException: Size of data received by IntegerDeserializer is not 4
There was a problem hiding this comment.
If it is a per-thread metric, I'd suggest we pre-register them at the beginning of the application. This way some other tools like JMXTool do not need to wait for the object name to show up. WDYT?
There was a problem hiding this comment.
Sounds good.
I meant to make a comment before you read this to say that there had been a concern in the discussion about having metrics reported from processor nodes (when the proposal was at the node level) that would never actually skip records, thereby polluting the metrics. I thought I'd throw the lazy registration pattern in just to see what you all thought.
I'll switch it back to pre-registration.
|
Jenkins failures are relevant to |
|
Ah, thanks, by the time I went to look at the last failure, the logs were already gone. |
|
I took the liberty of resolving all my ide warnings for the test files. Let me know if I went too far, and I can revert them. |
|
Checkstyle complained because I explicitly imported log4j, which is appropriate in this case. I isolated the usage to a "testutils" package, so I could allow the usage without allowing it for all of "stream.processor.internals". |
There was a problem hiding this comment.
A few meta comments:
- We should update the docs/upgrade-guide section, emphasizing that users who monitor based on the old metrics will be impacted (it is not compatible), and what they should start monitoring instead in the new version.
Also, we need to mention the new APIs added to TopologyTestDriver.
- We did not include any other places to record
skippedRecordsSensoryet. However, the added unit test still passed, I'm not sure why it is the case (skippedRecordsSensorwas not recorded when it happened yet)?
There was a problem hiding this comment.
Same here, we can get rid of String.format.
There was a problem hiding this comment.
We are stripping the prefix for this sensor: is it intentional? Note that for JMX reporter, the sensor name would not be included in any fields.
There was a problem hiding this comment.
skippedRecordsSensor should not be null, right?
There was a problem hiding this comment.
it was when I made it lazy. I've fixed it.
There was a problem hiding this comment.
nit: flattening to a very long single line, is it intentional?
There was a problem hiding this comment.
Ditto below and in other tests like testPauseResume
There was a problem hiding this comment.
it was not. I've fixed it.
There was a problem hiding this comment.
Why do we remove this sensor?
There was a problem hiding this comment.
Should we record the thread-level skipped record sensor here?
There was a problem hiding this comment.
heh, what a coincidence! I think so, and that's actually part of the motivation for this change I'm proposing to the metrics.
There was a problem hiding this comment.
Hmm.. I did not see we have recorded the sensor for deserialization error here, why this test passed?
There was a problem hiding this comment.
during thread.runOnce(-1);, it'll encounter an exception "asdfasdfasdf" as an integer and increment the metric.
There was a problem hiding this comment.
I disabled these tests because part of their function is to verify the number of metrics we register. This currently fails because we're registering a lot more metrics. If we decide to go with this overall strategy, I'll rethink these tests.
There was a problem hiding this comment.
Made a pass over this commit only. I think it is a good idea to encapsulate the sensors into the metrics so we do not pass individual sensors around.
One concern I had though, is that by moving StreamsMetricsThreadImpl to StreamsMetricsImpl, we are effectively wrapping the thread-level metrics into the InternalStreamsMetrics since the exposed APIs like taskCreatedSensor should be thread-level only (that makes thinking, should InternalStreamsMetrics just be ThreadMetrics)?
But for other layers, the awkward thing is that, for example, in TaskMetrics it has a reference of the total thread-level metrics which it does not really need: all it needs is the functionality to register more sensors into the underlying metrics registry.
So I'm wondering instead of make InternalStreamsMetrics extends StreamsMetrics. Could we:
-
make
InternalStreamsMetricsan independent interface with those thread-level metrics (we can rename it toThreadMetricsto clarify). -
let
StreamsMetricsThreadImplimplementThreadMetrics, while holding a reference to theStreamsMetricsregister the sensors. -
Similarly for
TaskMetrics,CacheMetricsandProcessorNodeMetrics, we do the same: each exposing its sensors as an API, while the corresponding impl class to get a reference of theStreamMetricsto call itsaddXXSensoretc functions.
WDYT?
There was a problem hiding this comment.
Just implements InternalStreamsMetrics should be sufficient, since InternalStreamsMetrics extends StreamsMetrics?
There was a problem hiding this comment.
Ah, yeah, in an earlier pass they were independent interfaces.
|
@guozhangwang I think you're right. For Streams, the thread level is the most global scope we have atm. I think what you're pointing out is that I've conflated the global scope with the thread scope. Ideally, these would be two separate scopes. Let me refactor a bit more, and see what you think. |
bbejeck
left a comment
There was a problem hiding this comment.
I've left just a couple of minor comments.
Overall I'm in favor of the change since this avoids the need to pass sensors around and returning values for metrics that can be captured where the event occurs.
But I agree with @guozhangwang on naming and structure suggested for the metrics refactoring proposed here.
There was a problem hiding this comment.
For testLatencyMetrics and testThroughputMetrics maybe use @Ignore instead ? Not a big deal but by getting an ignored test count there's a better chance these two tests won't fall through the cracks.
There was a problem hiding this comment.
ah, that's how you do it. I tried @Test(ignore=true) like testng, but that obviously doesn't work...
There was a problem hiding this comment.
For the task.addRecords with a long list of ConsumerRecord<> seems like the only difference with each record is the offset. Maybe create a method that takes an int[] with offsets and returns a List<ConsumerRecord>?
Actually I'm arguing that |
mjsax
left a comment
There was a problem hiding this comment.
Overall looks good. Couple of nits and questions.
There was a problem hiding this comment.
nit: add final to the parameters to cleanup code "on the side"
There was a problem hiding this comment.
when would skippedRecordsSensor be null? (I know this is just "move" code but still wondering why we need this)
There was a problem hiding this comment.
It won't. That's an artifact that I need to fix.
There was a problem hiding this comment.
Did this slip? Or did you leave it intentionally?
There was a problem hiding this comment.
nit: partitions -> partitionsForChangelog
|
@guozhangwang and @bbejeck about that experimental commit, I've decided to ditch it and implement the KIP with minimal changes to the structure of the metrics. I think I'd like to submit a KIP to alter the metric registration strategy we're employing later on, but I don't want to pollute KIP-274. |
|
I don't understand why these tests are failing. The message says: but line 804 in StreamTaskTest is: Retest this, please. |
|
I see (line 804) -- this makes sense, it should be |
|
Ah, it was because Jenkins (surprisingly) merges with trunk before testing. Also, there was an undetected merge conflict, resulting in the broken code. I've rebased and corrected it. Once the tests pass for me, I'll push again. |
There was a problem hiding this comment.
It's a private method with an unused return value. Making it void helps the reader to understand the code without having to trace through usages.
There was a problem hiding this comment.
@vvcephei @bbejeck @mjsax I'd suggest in the future try to only piggy-back different changes into the same PR if we think they are either correlated or if they are really trivial. Having a single PR mingled with multiple changes has several drawbacks:
- It makes git history a bit harder to trace: think, "git blame" would be tricker to reason.
- It tends to generate bigger PRs than necessary, making reviewer less willing to start working on them :P
- If multiple rounds of reviews are needed, even requiring major code refactoring, it will surprisingly introduce regressions during those iterations as by-products of the multiple changes.
There was a problem hiding this comment.
Nit: do we need to [] around each value? [] is used for collections or list -- might be confusing to add them?
There was a problem hiding this comment.
Good question. I have developed the habit of delimiting variables in log messages, as it disambiguates the structure of the message for the reader. Without delimiters, there are several edge cases that would make the log message difficult to read.
For example, if the key were "value" and the value were "" with the old format, you get:
Error sending records (key value value timestamp 1234)
Whereas, if the key were "" and the value were "value", you get
Error sending records (key value value timestamp 1234)
The only difference between these strings is where the extra space is. With delimiters, you have:
Error sending records (key=[value] value=[] timestamp=[1234])
Error sending records (key=[] value=[value] timestamp=[1234])
It's the kind of thing that saves people from #1 making a bad assumption about the nature of the problem and burning hours before they realize their mistake, or #2 being unable to clearly understand the error message and having to load it in a debugger just to understand what the values of the arguments actually are.
It sounds like your concern is about the ambiguity of [] as delimiters, since they already indicate a list. Can we keep delimiters but pick a different character? Other paired delimiters are <> and {}, and "" and '' also come to mind. WDYT?
There was a problem hiding this comment.
I personally thank, that having = (without []) is good enough as the = makes it clear:
Error sending records (key=value value= timestamp=1234)
Thus, it's not ambiguous to me (I agree that having no delimiter at all would be bad).
It's just that I like uniform formatting, and this would introduce a new style -- I am fine with change to this style, but we should agree on one style and rewrite code (on the side) if it does not fit the 'style guide'.
There was a problem hiding this comment.
I tend to prefer key=[value], but I do not have a scientific reason for that: I just feel it is more "vivid" :P
There was a problem hiding this comment.
@mjsax I can dig the desire to have uniform style on log messages. I'll also point out that the logs are part of the public API, so we can't just go terraform them willy-nilly, but instead we'd have to change them only in scope of the relevant KIPs, which makes it difficult to change, or even establish, a log style.
Nevertheless, if we don't already have a clear style for streams logs, I'll advocate for some kind of enclosing delimiter on substitutions. I continue to agree that square brackets are confusing w.r.t. common List#toString() formats, so I think we should agree on a different enclosing delimiter.
I agree that = is better than nothing, but it's still ambiguous when the substitution is 0 or more whitespace characters, while [] vs [ ] gives you more of a clue. No choice here is going to be perfect, but my experience is that this format saves enough debugging time to be worth the visual noise.
There was a problem hiding this comment.
if we don't already have a clear style for streams logs
We never discussed this explicitly; it's just a matter of fact that we use key=value so far from what I can remember. The question is, how much we gain if we start to rewrite to a different format and how much work it it.
With regard to ambiguity: you can always construct an (academic?) example for which any formatting strategy "break" and is ambiguous... If we agree on key=[value] I am fine with it. Still not sure, if we gain much (but if you think we do, it's fine with me to change)
There was a problem hiding this comment.
I also prefer key=[value], but can't say it's for any specific reason other than personal preference.
There was a problem hiding this comment.
Did this slip? Or did you leave it intentionally?
There was a problem hiding this comment.
Adding this implies, that we have to maintain the same code twice. Should we extract this into some internal method that we can call here to avoid code duplication?
What about other thread-level metrics?
There was a problem hiding this comment.
I've been mulling over the same thing, and that was part of what I was trying to achieve with my experiment before. I think I have a better solution now, so maybe you can take another look after my next update and see what you think.
There was a problem hiding this comment.
Will do, after you pushed an update :)
There was a problem hiding this comment.
For my own education: what is this? (btw: can we remove this below?)
There was a problem hiding this comment.
we can remove this below.
The WeakerAccess inspection tells you that it's possible to restrict the access scope of cancel(). I think this particular case was warning me that cancel() could be package-private instead of public. But the static analyzer can only look at the code in the project. We know that we do want the method to be public, so I added a supression for this inspection. An alternative would be to write black-box tests in a different package (just like real user tests would be), and the static analyser wouldn't warn us anymore, since it would have an example of a usage requiring public access.
bbejeck
left a comment
There was a problem hiding this comment.
Overall looks good, just a couple of additional minor comments.
There was a problem hiding this comment.
nit: I realize this was pre-existing in a single line, but since there are several parameters, maybe put each param on its own line.
There was a problem hiding this comment.
super nit: what about assertTrue((Double)metrics.metric(skippedRateMetric).metricValue() > 0.0); however, I don't have a strong opinion in this one.
There was a problem hiding this comment.
That also works, but assertNotEquals is a little nicer in that it'll print the actual value on failure, whereas assertTrue only tells you that it was false on failure. I suppose I could add a utility method assertGreater that prints the values on failure, but in this case, I'm really just making sure that the metric got moved. I don't care that much to assert what it got moved to, or I would override the time implementation and assert the exact expected value.
guozhangwang
left a comment
There was a problem hiding this comment.
Thanks for the PR. Overall it looks reasonable to me. I have only minor detailed comments, plus the meta comment about follow-up refactoring and the docs changes: 1) we should update upgrade-guide to warn users their monitoring needs to be updated, 2) in monitoring we'd better update the skipped records to list possible reasons it will be recorded.
There was a problem hiding this comment.
I'm do not feel very comfortable to define the metrics name in scattered places, because it means whenever we'll update the name we have to remember to update all the places (for this sensor the other place we declared it is
skippedRecordsSensor = metrics.sensor(prefix + ".skipped-records", Sensor.RecordingLevel.INFO);
skippedRecordsSensor.add(createMeter(metrics, new Sum(), "skipped-records", "skipped records"));
So which line gets called first, it will create the sensor, while keeping the other just as an no-op.
), and that's why I liked @vvcephei 's proposal for wrapping the sensor names in the leveled metrics, and passing those metrics across different modules than re-declaring the sensors in different places.
This makes me feel more urgent to do the refactoring of the metrics hierarchy.
There was a problem hiding this comment.
I'm about to push a commit to put the skipped-records sensor in particular in a common place, since it winds up getting accessed from so many different places in the code. I'm hoping that will be good enough for now, and we can seek an elegant enclosing-scope metrics implementation in the future.
There was a problem hiding this comment.
I tend to prefer key=[value], but I do not have a scientific reason for that: I just feel it is more "vivid" :P
There was a problem hiding this comment.
Again, if we could pass around the threadMetrics here, it will make the code more readable: we can make it very clear at which places we record some task metrics like taskMetrics.sensorA.record() and where do we record thread-level metrics like threadMetrics.skippedRecordsSensor.record().
But I think it is better to be left as a follow-up PR as this one is already pretty big.
There was a problem hiding this comment.
Yeah, I keep getting wrapped around the axle thinking about stuff like this. Hopefully, I'll be able to deliver a reasonable implementation for this PR, and I'll continue to mull about a way to pass the right metric context around the code base.
guozhangwang
left a comment
There was a problem hiding this comment.
Another meta comment: in KStreamAggregate, KStreamReduce, when either the key or value is null, we also skipped the record, should we record this as well since in many places it is not an expected event?
|
@guozhangwang I'm currently working on adding metrics there. I'm also adding warning logs, as it's totally silent right now. |
|
@guozhangwang and regarding this:
Ack. I might do that in a follow-up PR (under the same Jira/KIP) to keep the LOC in this PR lower. |
|
I've rebased and pushed the latest changes. I still need to add tests for the processors' metrics, but this change is otherwise pretty much where I want this PR to be. Note that I rebased and put the change to Also, reminder that we should agree on a log style. I left |
|
Ok, @bbejeck @mjsax @guozhangwang , This PR is ready for another pass. I have completed all code and tests. Docs will follow in another PR. Please comment on:
Thanks, all. |
|
retest this please |
guozhangwang
left a comment
There was a problem hiding this comment.
I left some comments on the high-level approach. Will dig into details after we have agreed on the proposal itself.
There was a problem hiding this comment.
What's the purpose of keep track of the metric names? If it is for preventing double-registering, I think relying on maintaining the metrics name inside the sensor would not always work, since multiple sensors would be added into the metrics registry, and we still cannot prevent different sensors trying to register the same metrics.
There was a problem hiding this comment.
I have pulled this change into a separate PR: #4853
The intent is to make it a no-op if you add the same metric to the same sensor twice, as opposed to the current behavior, in which the registry.registerMetric(metric) throws an exception if the metric is already registered.
With this change, you'll still get an exception if the metric is already registered in another sensor, but if it's already in the same sensor, you just get a no-op success.
There was a problem hiding this comment.
We do not need the non-arg constructors since it will be defined by default.
There was a problem hiding this comment.
This privatizes the constructor, guaranteeing that the class cannot be instantiated. It's a way of enforcing that the class be used only for its static members.
There was a problem hiding this comment.
To do this properly, though, the class should also be final. I'll make that change.
There was a problem hiding this comment.
No callers seem to provide any non-empty tags?
There was a problem hiding this comment.
It's laying the groundwork for a future change in callers can compose thread-level tags, task-level tags, etc.
There was a problem hiding this comment.
If we always create the skipped record sensor upon creating the thread, then we should always get the sensor right? If that case, should we simply throw if the getSensor returns null?
There was a problem hiding this comment.
This method is used to idempotently create or retrieve the sensor. In other words, the mechanism by which we create the sensor when we create the thread is that it calls this method, and the sensor is null, so it creates it.
You're correct in that if we do that, then all other usages will just return the existing sensor.
I'm not sure I see the value in separating creation from retrieval so that we can throw an exception if you retrieve it without creating it first.
There was a problem hiding this comment.
Thanks for the explanation, that makes sense.
There was a problem hiding this comment.
This is a meta comment:
I think we have seen two approaches here:
-
Pass along the metrics objects across different modules (in some classes, we will pass multiple metrics objects for different levels, like threadMetrics and taskMetrics) in order to record their sensors.
-
In the current PR: only pass alone the metrics registry (i.e. the
Metricsobject) along different modules, but standardize the sensor name construction, and get the sensor by its raw name directly whenever necessary to record the sensor.
I am slightly in favor of the second one since we could pass long fewer parameters, i.e. only a single Metrics object which can be accessed 1) from ProcessorContext, 2) in multiple internal classes.
There was a problem hiding this comment.
This is a detailed comment: what's the rantionale of naming it CommonStreamsMetrics? Is it for thread-level metrics only? I.e. should we just move this static function into ThreadMetrics?
There was a problem hiding this comment.
Yeah, I'm still undecided on whether approach 1 or 2 is better. But I did decide that I don't want to make a call on it in this PR. If you'd like me to resolve this sooner rather than later, I can follow up immediately with another PR to reorganize the metrics.
The reason I pulled skipped-records out into a CommonStreamsMetrics class is that that metric is common across all components in Streams. It's accessed by both our framework-level code and also by the user-space DSL-provided processors. Thus, it needs to live in a spot where all those components have visibility on it.
There's a distinction between metrics that are aggregated at the thread level and metrics that belong to StreamThread. It'll be difficult to really do a good job in this PR with that distinction, though, without refactoring the whole metrics hierarchy. Since we're already over 2,000 LOC, I'd really like to move such a refactoring to another PR.
What I have done in this PR is as minimal as I can manage to expose the skipped-records sensor to our Processors.
bbejeck
left a comment
There was a problem hiding this comment.
overall LGTM, just left some additional comments.
There was a problem hiding this comment.
I also prefer key=[value], but can't say it's for any specific reason other than personal preference.
There was a problem hiding this comment.
oops. I put it there when I needed a place for a breakpoint. Sorry!
There was a problem hiding this comment.
The hasItem matcher is new to me, nice one!
There was a problem hiding this comment.
Do we need a separate class for this? We could add the getMetricByName method to StreamsTestUtils instead, additionally, it doesn't access anything package private so it should be fine to make the method public
There was a problem hiding this comment.
I can move it to StreamTestUtils if you like.
I made getMetricByName package-private to prevent other code from calling it, since it's intended for these tests only. Making it public would open this method up to be called beyond the intended scope. I'd rather defer that until we have a use case we think calls for broadening the scope.
There was a problem hiding this comment.
meant to say this before, nice addition!
There was a problem hiding this comment.
Thanks! It's predicated on us using log4j as the implementation for slf4j in the unit tests, so if that changes, we'll have to put in a different log appender. But that seems unlikely, and it's handy to have this in the mean time.
|
Huh, that's a new one. It looks like (aside from the KafkaAdminTest continuing to flake out), the tests failed because the Jenkins worker ran out of disk! I'll wait until the last job completes before starting them again. I've rebased this PR on trunk now that #4853 is merged. I still have a few nits to clean up. I'll notify again when I'm ready for final reviews. |
|
Meta comment: please update the PR title with the JIRA number |
There was a problem hiding this comment.
Override to refine the type from StreamMetrics to StreamMetricsImpl in support of internal usages.
There was a problem hiding this comment.
This is an internal class, and this javadoc doesn't say anything that the method signature doesn't say. I added a new constructor and changed the existing one, so I just removed the doc rather than updating it.
There was a problem hiding this comment.
This existed only so a test could override it. Instead, I added a constructor arg for the test to pass and removed this method.
There was a problem hiding this comment.
I refactored these to flatten the metric definition, since it was super hard to figure out what metrics were actually being created.
Maybe you can forgive me for this because I actually found a bug: The description of the total metrics say that it counts the number of calls, but it previously summed the recorded values. (was via createMeter -> new Meter -> new Total)
There was a problem hiding this comment.
defining and providing skippedRecordsSensor here now, since it's now needed in contexts where the implementation is not a StreamThreadMetricsImpl.
There was a problem hiding this comment.
Similar to the metrics in StreamThread, by getting rid of the Meter and flattening these metrics, I realized that the total computation was incorrectly a Total rather than a Count.
There was a problem hiding this comment.
Not sure I understand this comment: it was indeed defined as a Count() before as well right?
There was a problem hiding this comment.
No, it was previously:
sensor.add(new Meter(new Count(), rateMetricName, totalMetricName));
But the Count there is only used for updating the rate. Here's the Metric constructor:
public Meter(SampledStat rateStat, MetricName rateMetricName, MetricName totalMetricName) {
this(TimeUnit.SECONDS, rateStat, rateMetricName, totalMetricName);
}
public Meter(TimeUnit unit, SampledStat rateStat, MetricName rateMetricName, MetricName totalMetricName) {
this.total = new Total();
this.rate = new Rate(unit, rateStat);
this.rateMetricName = rateMetricName;
this.totalMetricName = totalMetricName;
}
You can see that it's using Total for the "total" metric, which I guess makes sense given the parameter name. But according to the description of our "total" metric, what we really wanted to do was keep a count of occurrences, which should be a Count stat.
There was a problem hiding this comment.
I see. That makes sense.
I think it was not introducing any issue only because we only call that sensor.record() not sensor.record(n) so Count and Total are actually the same: record() is the same as record(1), but I agree that it should really be Count, to avoid any potential bugs.
There was a problem hiding this comment.
These metrics never get removed. Is that ok?
There was a problem hiding this comment.
Good point, let's add a TODO marker and remove them in a follow-up PR: so we do not drag too long on this one.
There was a problem hiding this comment.
Ok, when I took another look, I found that hitRatioSensor does get removed, but its parent doesn't. Also neither sensors have scoped names, so every record() will actually update all hit ratio metrics for all caches.
That seems like a bigger deal, so I've already prepared a follow-up PR. I'll send it next once this one is merged.
There was a problem hiding this comment.
This is required per the docs, but we previously only added it in production code paths. Now we add it in all code paths.
There was a problem hiding this comment.
This is required per the docs, but we previously only added it in production code paths. Now we add it in all code paths.
There was a problem hiding this comment.
The skipped records metrics are now always present. Rather than updating the hard-coded value, I did this to make the test less brittle.
There was a problem hiding this comment.
I replaced the override-with-capture strategy in this test with just a regular StreamsMetrics and verifying the invocation by checking that the total metric is == 1.
There was a problem hiding this comment.
"virtual" just in case people go looking for the actual thread in the thread dump. I also thought about using Thead.currentThread(), but it wouldn't necessarily be the same thread when the tests run.
There was a problem hiding this comment.
same thinking regarding "virtual"
|
ok, @guozhangwang @mjsax @bbejeck , I believe this is ready for final review. I've made a pass over it to make sure the diff is clean and to comment on the rationale of some of the choices. The diff is still quite long, but it's mostly because of all the processors that now record skipped metrics and the corresponding tests. |
guozhangwang
left a comment
There was a problem hiding this comment.
LGTM except an comment regarding
Similar to the metrics in StreamThread, by getting rid of the Meter and flattening these metrics, I realized that the total computation was incorrectly a Total rather than a Count.
| * @param sensor Sensor to record value. | ||
| */ | ||
| void measureLatencyNs(final Time time, final Runnable action, final Sensor sensor) { | ||
| public void measureLatencyNs(final Time time, final Runnable action, final Sensor sensor) { |
There was a problem hiding this comment.
Why this need to be public now?
There was a problem hiding this comment.
Because I moved StreamsMetricsImpl to the ...internal.metrics package, but I took a look, and it was only used (properly) by ProcessorNode.
StreamTask also used it, but only by specifically wrapping the operation in a Runnable and passing it to the metric to immediately be run.
I've added 18358f3 to simplify StreamTask's usage to not need this method, and move the method to ProcessorNode.
There was a problem hiding this comment.
Good point, let's add a TODO marker and remove them in a follow-up PR: so we do not drag too long on this one.
| this.logPrefix = String.format("task [%s] ", streamTaskId); | ||
| this.log = logContext.logger(getClass()); | ||
| this.productionExceptionHandler = productionExceptionHandler; | ||
| this.skippedRecordsSensor = skippedRecordsSensor; |
There was a problem hiding this comment.
I guess my previous comment was a bit misleading :P Actually I'm not against the CommonStreamsMetrics and ThreadMetricsConventions classes, but I think we could have one such class for each different layer than having a common class, for the reason I mentioned before. But since you have removed it I'm also fine with passing along the sensors as well.
We can consider which one is better in the near future and if we can do another code refactoring, but let's not block on this PR for too long.
| removeSensor(tasksClosedSensor); | ||
| removeSensor(skippedRecordsSensor); | ||
|
|
||
| removeSensor(skippedRecordsSensor()); |
There was a problem hiding this comment.
Hmm.. why we create the sensor in StreamsMetricsmpl while removing it in StreamsMetricsThreadImpl? It seems a bit inconsistency.. could we still create in StreamsMetricsThreadImpl, and let StreamsMetricsImpl to get a hold of the sensor object assuming it is already created then (i.e. set sensor == null in constructor, and in skippedRecordsSensor(): if (sensor == null) try get it from the metrics)?
There was a problem hiding this comment.
I tried that, but it wound up making things messier than I expected (because there are other callers to StreamsMetricsImpl, and because it makes the ownership of this sensor ambiguous).
Instead, I added 36c065f, which gives SMI the ability to remove its own sensors, and then I called to it from the two places (StreamThread and GlobalStreamThread) that actually need to unload metrics when they shut down.
WDYT?
There was a problem hiding this comment.
I did not completely get your explanation re: because there are other callers to StreamsMetricsImpl, and because it makes the ownership of this sensor ambiguous. Could you elaborate a bit more?
There was a problem hiding this comment.
Sorry; I misread your suggestion. I thought you wanted the StreamsMetricsImpl to take the sensor as a constructor argument.
Aside from the StreamThread (via StreamThreadMetricsImpl), several other classes directly invoke the StreamsMetricsImpl constructor and thus obtain a StreamsMetricsImpl that is not a StreamThreadMetricsImpl. Namely, GlobalStreamThread, MockProcessorContext, and TopologyTestDriver.
When the code paths downstream of these points need to record a skipped record, they will get a null sensor back. It wouldn't be possible to get it from the Metrics registry at that point, though, because the skipped records sensor is scoped by thread (or "virtual" thread for the test-utils), and the sensor would never have been created for GlobalStreamThread, MockProcessorContext, or TopologyTestDriver. So the only way to get it at that point is to have either the caller of the SMI constructor or the SMI itself create the sensor (either at construction or at call-time).
The previous implementation with the public static getter was effectively saying that the one who wants it in a particular context first creates it, but it's problematic because no-one owns it. And indeed, in my implementation, the sensor never got destroyed. In practice I think it's not a huge deal because I'm sure it's rare for a streams app to shut down and start up again with the same Metrics registry, and I think the threads live as long as the app. But still, we have this model of unloading metrics when they're out of scope, and I think it's a good one.
So that brings us to the current implementation. In the current implementation, the skipped records metric is clearly owned by the StreamsMetricsImpl, which it may as well be, since that is the smallest scope in which it's needed. It's the first metric to be owned by SMI, so I had to create a removeAll method, and make sure it's invoked in the right places. But that seems appropriate; every other scope that owns metrics has such a method.
There was a problem hiding this comment.
Not sure I understand this comment: it was indeed defined as a Count() before as well right?
|
@guozhangwang Thanks for the review! I added 18358f3 and 36c065f in response to your comments. I also have a next PR queued up for after this one is merged (in response to our concerns about NamedCacheMetrics). Please let me know what you think! |
bbejeck
left a comment
There was a problem hiding this comment.
overall looks good, just have a few comments.
There was a problem hiding this comment.
nit: Just thinking if this change is necessary as NodeMetrics is an internal class so a cast here from ProcessorContext to InternalProcessorContext should not be a big deal and keeps ProcessorNode more generic. EDIT: NM read a comment below about using type system and I agree.
There was a problem hiding this comment.
This is just used in one spot. It's easier to read the code if the log message is located at the spot where it gets logged rather than at the top of the file.
In earlier versions of Java, code like this was beneficial for performance, since the string is an object that can just get allocated and instantiated once statically, rather than dynamically on every invocation. But nowadays, the compiler and JIT compiler are smarter than that, so there really no benefit to coding this way, and you still pay the comprehensibility cost of having to follow the indirection.
I didn't want to make it a "thing", though, so I only inlined the message I needed to change.
There was a problem hiding this comment.
nit: shouldLogAndMeterOnSkippedRecords -> shouldLogAndMeterOnSkippedRecordsWithNullValue ?
There was a problem hiding this comment.
Line 503 the JavaDoc should change as the constructor for StreamsMetricsImpl only takes Metrics and a String parameter.
There was a problem hiding this comment.
Ah, good eye. I might just ditch the Javadoc, since it's an internal class and its function is pretty obvious.
There was a problem hiding this comment.
Maybe consider replacing Stack with Deque as Stack is synchronized and ownedSensors only adds in the constructor and removes values in synchronized block already.
There was a problem hiding this comment.
Ooh, good call. I will do that and probably avoid using Stack again.
|
retest this please |
* unify skipped records metering * log warnings when things get skipped * tighten up metrics usage a bit
|
Addressed Bill's comments and rebased. |
* unify skipped records metering * log warnings when things get skipped * tighten up metrics usage a bit ### Testing strategy: Unit testing of the metrics and the logs should be sufficient. Author: John Roesler <john@confluent.io> Reviewers: Bill Bejeck <bill@confluent.io>, Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com> Closes apache#4812 from vvcephei/kip-274-streams-skip-metrics
* unify skipped records metering * log warnings when things get skipped * tighten up metrics usage a bit ### Testing strategy: Unit testing of the metrics and the logs should be sufficient. Author: John Roesler <john@confluent.io> Reviewers: Bill Bejeck <bill@confluent.io>, Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com> Closes apache#4812 from vvcephei/kip-274-streams-skip-metrics
Testing strategy:
Unit testing of the metrics and the logs should be sufficient.
Committer Checklist (excluded from commit message)