flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (FLINK-4246) Allow Specifying Multiple Metrics Reporters
Date Fri, 22 Jul 2016 15:24:20 GMT

    [ https://issues.apache.org/jira/browse/FLINK-4246?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15389671#comment-15389671
] 

ASF GitHub Bot commented on FLINK-4246:
---------------------------------------

Github user aljoscha commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2285#discussion_r71896724
  
    --- Diff: flink-core/src/main/java/org/apache/flink/metrics/MetricRegistry.java ---
    @@ -75,77 +78,92 @@ public MetricRegistry(Configuration config) {
     		this.delimiter = delim;
     
     		// second, instantiate any custom configured reporters
    -		
    -		final String className = config.getString(ConfigConstants.METRICS_REPORTER_CLASS, null);
    -		if (className == null) {
    +		this.reporters = new ArrayList<>();
    +
    +		final String definedReporters = config.getString(ConfigConstants.METRICS_REPORTERS_LIST,
null);
    +
    +		if (definedReporters == null) {
    +			// no reporters defined
     			// by default, don't report anything
     			LOG.info("No metrics reporter configured, no metrics will be exposed/reported.");
    -			this.reporter = null;
     			this.executor = null;
    -		}
    -		else {
    -			MetricReporter reporter;
    -			ScheduledExecutorService executor = null;
    -			try {
    -				String configuredPeriod = config.getString(ConfigConstants.METRICS_REPORTER_INTERVAL,
null);
    -				TimeUnit timeunit = TimeUnit.SECONDS;
    -				long period = 10;
    -				
    -				if (configuredPeriod != null) {
    -					try {
    -						String[] interval = configuredPeriod.split(" ");
    -						period = Long.parseLong(interval[0]);
    -						timeunit = TimeUnit.valueOf(interval[1]);
    +		} else {
    +			// we have some reporters so
    +			String[] namedReporters = definedReporters.split(",");
    +			for (String namedReporter : namedReporters) {
    --- End diff --
    
    In that case the array should be empty and we should not enter the loop, right?


> Allow Specifying Multiple Metrics Reporters
> -------------------------------------------
>
>                 Key: FLINK-4246
>                 URL: https://issues.apache.org/jira/browse/FLINK-4246
>             Project: Flink
>          Issue Type: Improvement
>          Components: Metrics
>    Affects Versions: 1.1.0
>            Reporter: Aljoscha Krettek
>            Assignee: Aljoscha Krettek
>             Fix For: 1.1.0
>
>
> We should allow specifying multiple reporters. A rough sketch of how the configuration
should look like is this:
> {code}
> metrics.reporters = foo,bar
> metrics.reporter.foo.class = JMXReporter.class
> metrics.reporter.foo.port = 42-117
> metrics.reporter.bar.class = GangliaReporter.class
> metrics.reporter.bar.port = 512
> metrics.reporter.bar.whatever = 2
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Mime
View raw message