You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by tzulitai <gi...@git.apache.org> on 2018/01/22 15:14:57 UTC
[GitHub] flink pull request #5335: (master) [FLINK-8419] [kafka] Register metrics for...
GitHub user tzulitai opened a pull request:
https://github.com/apache/flink/pull/5335
(master) [FLINK-8419] [kafka] Register metrics for dynamically discovered Kafka partitions
## What is the purpose of the change
This PR fixes that offset metrics (i.e. current offset, committed offsets) were not registered for partitions that were dynamically discovered in the `FlinkKafkaConsumerBase`.
This version is targeted for merge to `master`.
Another version targeted for `release-1.4`, which does not include the new offset metrics added in #5214, will be separately opened.
## Brief change log
- 54f3cfd: Preliminary cleanup of the registration of the `KafkaConsumer` user scope metric group. This commit refactors that for better separation of concerns and lessen code duplication.
- bf1e4ce: Register offset metrics for new partitions in `addDiscoveredPartitions`
- 7e90a75: Minor hotfix to inappropriate access modifiers in the `AbstractFetcher`
## Verifying this change
No new tests were added.
By manually running a Flink job using the Kafka consumer and repartitioning the Kafka topic, you should be able to see metrics for the newly added partitions.
## Does this pull request potentially affect one of the following parts:
- Dependencies (does it add or upgrade a dependency): (yes / **no**)
- The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (yes / **no**)
- The serializers: (yes / **no** / don't know)
- The runtime per-record code paths (performance sensitive): (yes / **no** / don't know)
- Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper: (yes / **no** / don't know)
- The S3 file system connector: (yes / **no** / don't know)
## Documentation
- Does this pull request introduce a new feature? (yes / **no**)
- If yes, how is the feature documented? (**not applicable** / docs / JavaDocs / not documented)
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/tzulitai/flink FLINK-8419
Alternatively you can review and apply these changes as the patch at:
https://github.com/apache/flink/pull/5335.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 #5335
----
commit 54f3cfd3eb9f925266de37219ab56703a26795d6
Author: Tzu-Li (Gordon) Tai <tz...@...>
Date: 2018-01-22T12:36:20Z
[FLINK-8419] [kafka] Move consumer metric group registration to FlinkKafkaConsumerBase
This commit is a refactor to move the registration of the consumer
metric group (user scope "KafkaConsumer") to FlinkKafkaConsumerBase.
Previously, the registration was scattered around in Kafka
version-specific subclasses.
commit bf1e4ce73279a90d625cda66ee84c194d5ce3e34
Author: Tzu-Li (Gordon) Tai <tz...@...>
Date: 2018-01-22T13:14:44Z
[FLINK-8419] [kafka] Register metrics for dynamically discovered Kafka partitions
commit 7e90a753441b7a4af1dd400942d88fba7d0178df
Author: Tzu-Li (Gordon) Tai <tz...@...>
Date: 2018-01-22T13:17:54Z
[hotfix] [kafka] Fix inapproriate access modifiers in AbstractFetcher
----
---
[GitHub] flink pull request #5335: (master) [FLINK-8419] [kafka] Register metrics for...
Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:
https://github.com/apache/flink/pull/5335#discussion_r165022709
--- Diff: flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java ---
@@ -95,21 +95,19 @@ public Kafka09Fetcher(
watermarksPunctuated,
processingTimeProvider,
autoWatermarkInterval,
- userCodeClassLoader,
+ userCodeClassLoader.getParent(),
+ consumerMetricGroup,
useMetrics);
this.deserializer = deserializer;
this.handover = new Handover();
- final MetricGroup kafkaMetricGroup = metricGroup.addGroup(KAFKA_CONSUMER_METRICS_GROUP);
- addOffsetStateGauge(kafkaMetricGroup);
-
this.consumerThread = new KafkaConsumerThread(
LOG,
handover,
kafkaProperties,
unassignedPartitionsQueue,
- kafkaMetricGroup,
+ subtaskMetricGroup, // TODO: the thread should expose Kafka-shipped metrics through the consumer metric group, not subtask metric group
--- End diff --
so why aren't we passing the consumerMetricGroup here?
---
[GitHub] flink pull request #5335: (master) [FLINK-8419] [kafka] Register metrics for...
Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:
https://github.com/apache/flink/pull/5335
---
[GitHub] flink pull request #5335: (master) [FLINK-8419] [kafka] Register metrics for...
Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:
https://github.com/apache/flink/pull/5335#discussion_r165045598
--- Diff: flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/internal/Kafka09Fetcher.java ---
@@ -95,21 +95,19 @@ public Kafka09Fetcher(
watermarksPunctuated,
processingTimeProvider,
autoWatermarkInterval,
- userCodeClassLoader,
+ userCodeClassLoader.getParent(),
+ consumerMetricGroup,
useMetrics);
this.deserializer = deserializer;
this.handover = new Handover();
- final MetricGroup kafkaMetricGroup = metricGroup.addGroup(KAFKA_CONSUMER_METRICS_GROUP);
- addOffsetStateGauge(kafkaMetricGroup);
-
this.consumerThread = new KafkaConsumerThread(
LOG,
handover,
kafkaProperties,
unassignedPartitionsQueue,
- kafkaMetricGroup,
+ subtaskMetricGroup, // TODO: the thread should expose Kafka-shipped metrics through the consumer metric group, not subtask metric group
--- End diff --
Will address this as discussed in #5336, and then merge this.
---