[GitHub] flink pull request #2517: [FLINK-4564] [metrics] Delimiter should be configu...
Github user asfgit closed the pull request at: https://github.com/apache/flink/pull/2517 --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request #2517: [FLINK-4564] [metrics] Delimiter should be configu...
Github user ex00 commented on a diff in the pull request: https://github.com/apache/flink/pull/2517#discussion_r80927679 --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistry.java --- @@ -216,17 +244,20 @@ public ScopeFormats getScopeFormats() { * @param metricName the name of the metric * @param group the group that contains the metric */ - public void register(Metric metric, String metricName, MetricGroup group) { + public void register(Metric metric, String metricName, AbstractMetricGroup group) { try { if (reporters != null) { - for (MetricReporter reporter : reporters) { + for (int i = 0; i < reporters.size(); i++) { + MetricReporter reporter = reporters.get(i); if (reporter != null) { - reporter.notifyOfAddedMetric(metric, metricName, group); + FrontMetricGroup front = new FrontMetricGroup(i); + front.setReference(group); --- End diff -- If we need use `FrontMetricGroup` also for `unregister()`, do I need return list of `FrontMetricGroup` ? and we will avoid for `register()` >we will now create a new object every time we add a new metric. in this case --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request #2517: [FLINK-4564] [metrics] Delimiter should be configu...
Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/2517#discussion_r80886970 --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistry.java --- @@ -216,17 +244,20 @@ public ScopeFormats getScopeFormats() { * @param metricName the name of the metric * @param group the group that contains the metric */ - public void register(Metric metric, String metricName, MetricGroup group) { + public void register(Metric metric, String metricName, AbstractMetricGroup group) { try { if (reporters != null) { - for (MetricReporter reporter : reporters) { + for (int i = 0; i < reporters.size(); i++) { + MetricReporter reporter = reporters.get(i); if (reporter != null) { - reporter.notifyOfAddedMetric(metric, metricName, group); + FrontMetricGroup front = new FrontMetricGroup(i); + front.setReference(group); --- End diff -- The references must also be set within `unregister()`. --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request #2517: [FLINK-4564] [metrics] Delimiter should be configu...
Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/2517#discussion_r80886088 --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistry.java --- @@ -216,17 +244,20 @@ public ScopeFormats getScopeFormats() { * @param metricName the name of the metric * @param group the group that contains the metric */ - public void register(Metric metric, String metricName, MetricGroup group) { + public void register(Metric metric, String metricName, AbstractMetricGroup group) { try { if (reporters != null) { - for (MetricReporter reporter : reporters) { + for (int i = 0; i < reporters.size(); i++) { + MetricReporter reporter = reporters.get(i); if (reporter != null) { - reporter.notifyOfAddedMetric(metric, metricName, group); + FrontMetricGroup front = new FrontMetricGroup(i); --- End diff -- we will now create a new object every time we add a new metric. Instead, keep a single `FrontMetricGroup` instance in the registry, and add a `setIndex()` method to it that you call here along with `setReference()`. but, i just realized that without this you would end up with concurrency issues since multiple register calls can be active at the same time...I'll have to think about this for a bit. --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request #2517: [FLINK-4564] [metrics] Delimiter should be configu...
Github user ex00 commented on a diff in the pull request: https://github.com/apache/flink/pull/2517#discussion_r80499420 --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistry.java --- @@ -219,9 +249,16 @@ public ScopeFormats getScopeFormats() { public void register(Metric metric, String metricName, MetricGroup group) { --- End diff -- In this case I get error in the compile for extending class of MetricRegistry. For example org.apache.flink.runtime.metrics.groups.TaskMetricGroupTest.CountingMetricRegistry#register(Metric , String , MetricGroup) and org.apache.flink.runtime.metrics.groups.MetricGroupTest.ExceptionOnRegisterRegistry#register(Metric , String , MetricGroup) Cause by: method does not override or implement a method from a supertype It is simple tests classes, I can change methods signature in their. But will not be any problems in future because of this change in MetricRegistry? --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request #2517: [FLINK-4564] [metrics] Delimiter should be configu...
Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/2517#discussion_r80474584 --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistry.java --- @@ -219,9 +249,16 @@ public ScopeFormats getScopeFormats() { public void register(Metric metric, String metricName, MetricGroup group) { try { if (reporters != null) { - for (MetricReporter reporter : reporters) { + for (int i = 0; i < reporters.size(); i++) { + MetricReporter reporter = reporters.get(i); if (reporter != null) { - reporter.notifyOfAddedMetric(metric, metricName, group); + if (group instanceof AbstractMetricGroup) { --- End diff -- this will always be true. --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request #2517: [FLINK-4564] [metrics] Delimiter should be configu...
Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/2517#discussion_r80474552 --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistry.java --- @@ -219,9 +249,16 @@ public ScopeFormats getScopeFormats() { public void register(Metric metric, String metricName, MetricGroup group) { --- End diff -- if you change the signature to (Metric, String, AbstractMetricGroup) you would no longer need the cast on L257. --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request #2517: [FLINK-4564] [metrics] Delimiter should be configu...
Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/2517#discussion_r80473933 --- Diff: docs/monitoring/metrics.md --- @@ -280,6 +280,7 @@ Metrics can be exposed to an external system by configuring one or several repor - `metrics.reporter..`: Generic setting `` for the reporter named ``. - `metrics.reporter..class`: The reporter class to use for the reporter named ``. - `metrics.reporter..interval`: The reporter interval to use for the reporter named ``. +- `metrics.reporter..scope.delimiter`: The delimiter for to use for the identifier (default value use `metrics.scope.delimiter`) for the reporter named ``. --- End diff -- for to use -> to use --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request #2517: [FLINK-4564] [metrics] Delimiter should be configu...
Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/2517#discussion_r80480187 --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/FrontMetricGroup.java --- @@ -0,0 +1,129 @@ +/* + * 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.metrics.groups; + +import org.apache.flink.metrics.CharacterFilter; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.Histogram; +import org.apache.flink.metrics.Meter; +import org.apache.flink.metrics.MetricGroup; + +import java.util.Map; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * When need know position number for reporter in metric group + * + * @param reference to {@link AbstractMetricGroup AbstractMetricGroup} + */ +public class FrontMetricGroup> implements MetricGroup { + + protected A reference; + protected int index; --- End diff -- should be called reporterIndex. --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request #2517: [FLINK-4564] [metrics] Delimiter should be configu...
Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/2517#discussion_r80477622 --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/metrics/MetricRegistryTest.java --- @@ -280,4 +282,65 @@ public void testConfigurableDelimiter() { registry.shutdown(); } + + @Test + public void testConfigurableDelimiterForReporters() { + Configuration config = new Configuration(); + config.setString(ConfigConstants.METRICS_REPORTERS_LIST, "test1,test2,test3"); + config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test1." + ConfigConstants.METRICS_REPORTER_SCOPE_DELIMITER, "_"); + config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test1." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, TestReporter.class.getName()); + config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test2." + ConfigConstants.METRICS_REPORTER_SCOPE_DELIMITER, "-"); + config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test2." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, TestReporter.class.getName()); + config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test3." + ConfigConstants.METRICS_REPORTER_SCOPE_DELIMITER, "AA"); + config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test3." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, TestReporter.class.getName()); + + MetricRegistry registry = new MetricRegistry(config); + + assertEquals('.', registry.getDelimiter()); --- End diff -- introducing a static final char field for the global delimiter would improve readability. --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request #2517: [FLINK-4564] [metrics] Delimiter should be configu...
Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/2517#discussion_r80475991 --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/metrics/MetricRegistryTest.java --- @@ -280,4 +282,65 @@ public void testConfigurableDelimiter() { registry.shutdown(); } + + @Test + public void testConfigurableDelimiterForReporters() { + Configuration config = new Configuration(); + config.setString(ConfigConstants.METRICS_REPORTERS_LIST, "test1,test2,test3"); + config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test1." + ConfigConstants.METRICS_REPORTER_SCOPE_DELIMITER, "_"); + config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test1." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, TestReporter.class.getName()); + config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test2." + ConfigConstants.METRICS_REPORTER_SCOPE_DELIMITER, "-"); + config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test2." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, TestReporter.class.getName()); + config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test3." + ConfigConstants.METRICS_REPORTER_SCOPE_DELIMITER, "AA"); + config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test3." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, TestReporter.class.getName()); + + MetricRegistry registry = new MetricRegistry(config); + + assertEquals('.', registry.getDelimiter()); + assertEquals('_', registry.getDelimiter(0)); + assertEquals('-', registry.getDelimiter(1)); + assertEquals('.', registry.getDelimiter(2)); + assertEquals('.', registry.getDelimiter(3)); + assertEquals('.', registry.getDelimiter(-1)); + + registry.shutdown(); + } + + @Test + public void testConfigurableDelimiterForReportersInGroup() { + Configuration config = new Configuration(); + config.setString(ConfigConstants.METRICS_REPORTERS_LIST, "test1,test2,test3,test4"); + config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test1." + ConfigConstants.METRICS_REPORTER_SCOPE_DELIMITER, "_"); + config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test1." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, TestReporter8.class.getName()); + config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test2." + ConfigConstants.METRICS_REPORTER_SCOPE_DELIMITER, "-"); + config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test2." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, TestReporter8.class.getName()); + config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test3." + ConfigConstants.METRICS_REPORTER_SCOPE_DELIMITER, "AA"); + config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test3." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, TestReporter8.class.getName()); + config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test4." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, TestReporter8.class.getName()); + config.setString(ConfigConstants.METRICS_SCOPE_NAMING_TM, "A.B"); + + MetricRegistry registry = new MetricRegistry(config); + List reporters = registry.getReporters(); + ((TestReporter8)reporters.get(0)).expectedDelimiter = '_'; //test1 reporter + ((TestReporter8)reporters.get(1)).expectedDelimiter = '-'; //test2 reporter + ((TestReporter8)reporters.get(2)).expectedDelimiter = '.'; //test3 reporter, because 'AA' - not correct delimiter + //for test4 reporter use global delimiter + --- End diff -- are you sure that this test accurately detects errors? if the assert within notify() fails an exception is thrown, which however is never propagated but only logged. It would however be correct if you execute countCall++ after assertEquals(), as you only reach it for a correct delimiter. Then of course it should be renamed to something like "numCorrectDelimiters" or something. --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request #2517: [FLINK-4564] [metrics] Delimiter should be configu...
Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/2517#discussion_r80477644 --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/metrics/MetricRegistryTest.java --- @@ -280,4 +282,65 @@ public void testConfigurableDelimiter() { registry.shutdown(); } + + @Test + public void testConfigurableDelimiterForReporters() { + Configuration config = new Configuration(); + config.setString(ConfigConstants.METRICS_REPORTERS_LIST, "test1,test2,test3"); + config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test1." + ConfigConstants.METRICS_REPORTER_SCOPE_DELIMITER, "_"); + config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test1." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, TestReporter.class.getName()); + config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test2." + ConfigConstants.METRICS_REPORTER_SCOPE_DELIMITER, "-"); + config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test2." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, TestReporter.class.getName()); + config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test3." + ConfigConstants.METRICS_REPORTER_SCOPE_DELIMITER, "AA"); + config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test3." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, TestReporter.class.getName()); + + MetricRegistry registry = new MetricRegistry(config); + + assertEquals('.', registry.getDelimiter()); + assertEquals('_', registry.getDelimiter(0)); + assertEquals('-', registry.getDelimiter(1)); + assertEquals('.', registry.getDelimiter(2)); + assertEquals('.', registry.getDelimiter(3)); + assertEquals('.', registry.getDelimiter(-1)); + + registry.shutdown(); + } + + @Test + public void testConfigurableDelimiterForReportersInGroup() { + Configuration config = new Configuration(); + config.setString(ConfigConstants.METRICS_REPORTERS_LIST, "test1,test2,test3,test4"); + config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test1." + ConfigConstants.METRICS_REPORTER_SCOPE_DELIMITER, "_"); + config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test1." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, TestReporter8.class.getName()); + config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test2." + ConfigConstants.METRICS_REPORTER_SCOPE_DELIMITER, "-"); + config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test2." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, TestReporter8.class.getName()); + config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test3." + ConfigConstants.METRICS_REPORTER_SCOPE_DELIMITER, "AA"); + config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test3." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, TestReporter8.class.getName()); + config.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "test4." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, TestReporter8.class.getName()); + config.setString(ConfigConstants.METRICS_SCOPE_NAMING_TM, "A.B"); + + MetricRegistry registry = new MetricRegistry(config); + List reporters = registry.getReporters(); + ((TestReporter8)reporters.get(0)).expectedDelimiter = '_'; //test1 reporter --- End diff -- for clarity purposes we should explicitly set the expectedDelimiter here. --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request #2517: [FLINK-4564] [metrics] Delimiter should be configu...
Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/2517#discussion_r80479173 --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistry.java --- @@ -219,9 +249,16 @@ public ScopeFormats getScopeFormats() { public void register(Metric metric, String metricName, MetricGroup group) { try { if (reporters != null) { - for (MetricReporter reporter : reporters) { + for (int i = 0; i < reporters.size(); i++) { + MetricReporter reporter = reporters.get(i); if (reporter != null) { - reporter.notifyOfAddedMetric(metric, metricName, group); + if (group instanceof AbstractMetricGroup) { + FrontMetricGroup front = groups.get(i); --- End diff -- just realized that we can simplify this. We don't need multiple FrontMetricGroups, but only 1 on which we also set the index before notify(). --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request #2517: [FLINK-4564] [metrics] Delimiter should be configu...
Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/2517#discussion_r80481547 --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/FrontMetricGroup.java --- @@ -0,0 +1,129 @@ +/* + * 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.metrics.groups; + +import org.apache.flink.metrics.CharacterFilter; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.Histogram; +import org.apache.flink.metrics.Meter; +import org.apache.flink.metrics.MetricGroup; + +import java.util.Map; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * When need know position number for reporter in metric group --- End diff -- "Metric group which forwards all registration calls to a varaible parent metric group that injects a variable reporter index into calls to {@link org.apache.flink.metrics.MetricGroup#getMetricIdentifier(String)} or {@link org.apache.flink.metrics.MetricGroup#getMetricIdentifier(String, CharacterFilter)}. This allows us to use reporter-specific delimiters, without requiring any action by the reporter." --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request #2517: [FLINK-4564] [metrics] Delimiter should be configu...
Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/2517#discussion_r80242008 --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistry.java --- @@ -99,10 +99,17 @@ public MetricRegistry(Configuration config) { DelegatingConfiguration reporterConfig = new DelegatingConfiguration(config, ConfigConstants.METRICS_REPORTER_PREFIX + namedReporter + "."); final String className = reporterConfig.getString(ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, null); if (className == null) { - LOG.error("No reporter class set for reporter " + namedReporter + ". Metrics might not be exposed/reported."); + LOG.error("No reporter class set for reporter {}. Metrics might not be exposed/reported.",namedReporter); continue; } + String delimiterForReporter = reporterConfig.getString(ConfigConstants.METRICS_REPORTER_SCOPE_DELIMITER,defaultDelimiter); + if(delimiterForReporter.length()!=1){ + LOG.warn("Failed to parse delimiter for reporter {}, using global delimiter.",namedReporter); --- End diff -- and possibly the what was parsed. --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request #2517: [FLINK-4564] [metrics] Delimiter should be configu...
Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/2517#discussion_r80241061 --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/AbstractMetricGroup.java --- @@ -168,21 +168,40 @@ public String getMetricIdentifier(String metricName) { * @param filter character filter which is applied to the scope components if not null. * @return fully qualified metric name */ - public String getMetricIdentifier(String metricName, CharacterFilter filter) { - if (scopeString == null) { - if (filter != null) { - scopeString = ScopeFormat.concat(filter, registry.getDelimiter(), scopeComponents); - } else { - scopeString = ScopeFormat.concat(registry.getDelimiter(), scopeComponents); - } + public String getMetricIdentifier(String metricName,CharacterFilter filter) { + if (filter != null){ + scopeString = ScopeFormat.concat(filter, registry.getDelimiter(), scopeComponents); + } else { + scopeString = ScopeFormat.concat(registry.getDelimiter(), scopeComponents); } - - if (filter != null) { + if (filter != null){ return scopeString + registry.getDelimiter() + filter.filterCharacters(metricName); } else { return scopeString + registry.getDelimiter() + metricName; } } + + /** +* Returns the fully qualified metric name for reporter with certain number position in group, for example +* {@code "host-7.taskmanager-2.window_word_count.my-mapper.metricName"} +* +* @param metricName metric name +* @param filter character filter which is applied to the scope components if not null. +* @param indexReporter position number of reporter in group reporters +* @return fully qualified metric name +*/ + public String getMetricIdentifier(String metricName, CharacterFilter filter, int indexReporter) { + if (filter != null){ --- End diff -- missing space before ```{``` --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request #2517: [FLINK-4564] [metrics] Delimiter should be configu...
Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/2517#discussion_r80241086 --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/AbstractMetricGroup.java --- @@ -168,21 +168,40 @@ public String getMetricIdentifier(String metricName) { * @param filter character filter which is applied to the scope components if not null. * @return fully qualified metric name */ - public String getMetricIdentifier(String metricName, CharacterFilter filter) { - if (scopeString == null) { - if (filter != null) { - scopeString = ScopeFormat.concat(filter, registry.getDelimiter(), scopeComponents); - } else { - scopeString = ScopeFormat.concat(registry.getDelimiter(), scopeComponents); - } + public String getMetricIdentifier(String metricName,CharacterFilter filter) { + if (filter != null){ + scopeString = ScopeFormat.concat(filter, registry.getDelimiter(), scopeComponents); + } else { + scopeString = ScopeFormat.concat(registry.getDelimiter(), scopeComponents); } - - if (filter != null) { + if (filter != null){ return scopeString + registry.getDelimiter() + filter.filterCharacters(metricName); } else { return scopeString + registry.getDelimiter() + metricName; } } + + /** +* Returns the fully qualified metric name for reporter with certain number position in group, for example --- End diff -- the javadoc should say "Returns the fully qualified metric name using the configured delimiter for the reporter with the given index. --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request #2517: [FLINK-4564] [metrics] Delimiter should be configu...
Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/2517#discussion_r80240197 --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistry.java --- @@ -99,10 +99,17 @@ public MetricRegistry(Configuration config) { DelegatingConfiguration reporterConfig = new DelegatingConfiguration(config, ConfigConstants.METRICS_REPORTER_PREFIX + namedReporter + "."); final String className = reporterConfig.getString(ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, null); if (className == null) { - LOG.error("No reporter class set for reporter " + namedReporter + ". Metrics might not be exposed/reported."); + LOG.error("No reporter class set for reporter {}. Metrics might not be exposed/reported.",namedReporter); continue; } + String delimiterForReporter = reporterConfig.getString(ConfigConstants.METRICS_REPORTER_SCOPE_DELIMITER,defaultDelimiter); + if(delimiterForReporter.length()!=1){ + LOG.warn("Failed to parse delimiter for reporter {}, using global delimiter.",namedReporter); + delimiterForReporter="."; + } + this.delimiters.add(delimiterForReporter.charAt(0)); + --- End diff -- Please add the configured delimiter to the info log message when a reporter was instantiated. (near the end of the constructor) --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request #2517: [FLINK-4564] [metrics] Delimiter should be configu...
Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/2517#discussion_r80239440 --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistry.java --- @@ -72,14 +74,12 @@ public MetricRegistry(Configuration config) { } this.scopeFormats = scopeFormats; - char delim; - try { - delim = config.getString(ConfigConstants.METRICS_SCOPE_DELIMITER, ".").charAt(0); - } catch (Exception e) { - LOG.warn("Failed to parse delimiter, using default delimiter.", e); - delim = '.'; + String defaultDelimiter = config.getString(ConfigConstants.METRICS_SCOPE_DELIMITER, "."); + if(defaultDelimiter.length()!=1){ --- End diff -- these changes are unnecessary, renaming ```delimiter``` to ```globalDelimiter``` should be sufficient. --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request #2517: [FLINK-4564] [metrics] Delimiter should be configu...
Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/2517#discussion_r80238884 --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/AbstractMetricGroup.java --- @@ -168,21 +168,40 @@ public String getMetricIdentifier(String metricName) { * @param filter character filter which is applied to the scope components if not null. * @return fully qualified metric name */ - public String getMetricIdentifier(String metricName, CharacterFilter filter) { - if (scopeString == null) { - if (filter != null) { - scopeString = ScopeFormat.concat(filter, registry.getDelimiter(), scopeComponents); - } else { - scopeString = ScopeFormat.concat(registry.getDelimiter(), scopeComponents); - } + public String getMetricIdentifier(String metricName,CharacterFilter filter) { --- End diff -- these changes should be reverted. --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request #2517: [FLINK-4564] [metrics] Delimiter should be configu...
Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/2517#discussion_r80241027 --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/AbstractMetricGroup.java --- @@ -168,21 +168,40 @@ public String getMetricIdentifier(String metricName) { * @param filter character filter which is applied to the scope components if not null. * @return fully qualified metric name */ - public String getMetricIdentifier(String metricName, CharacterFilter filter) { - if (scopeString == null) { - if (filter != null) { - scopeString = ScopeFormat.concat(filter, registry.getDelimiter(), scopeComponents); - } else { - scopeString = ScopeFormat.concat(registry.getDelimiter(), scopeComponents); - } + public String getMetricIdentifier(String metricName,CharacterFilter filter) { + if (filter != null){ + scopeString = ScopeFormat.concat(filter, registry.getDelimiter(), scopeComponents); + } else { + scopeString = ScopeFormat.concat(registry.getDelimiter(), scopeComponents); } - - if (filter != null) { + if (filter != null){ return scopeString + registry.getDelimiter() + filter.filterCharacters(metricName); } else { return scopeString + registry.getDelimiter() + metricName; } } + + /** +* Returns the fully qualified metric name for reporter with certain number position in group, for example +* {@code "host-7.taskmanager-2.window_word_count.my-mapper.metricName"} +* +* @param metricName metric name +* @param filter character filter which is applied to the scope components if not null. +* @param indexReporter position number of reporter in group reporters +* @return fully qualified metric name +*/ + public String getMetricIdentifier(String metricName, CharacterFilter filter, int indexReporter) { --- End diff -- argument should be called ```reporterIndex``` --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request #2517: [FLINK-4564] [metrics] Delimiter should be configu...
Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/2517#discussion_r80240603 --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistry.java --- @@ -219,9 +235,19 @@ public ScopeFormats getScopeFormats() { public void register(Metric metric, String metricName, MetricGroup group) { try { if (reporters != null) { - for (MetricReporter reporter : reporters) { - if (reporter != null) { - reporter.notifyOfAddedMetric(metric, metricName, group); + for (int i= 0; i
[GitHub] flink pull request #2517: [FLINK-4564] [metrics] Delimiter should be configu...
Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/2517#discussion_r80239024 --- Diff: flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/groups/UnregisteredMetricsGroup.java --- @@ -114,4 +114,9 @@ public String getMetricIdentifier(String metricName) { public String getMetricIdentifier(String metricName, CharacterFilter filter) { return metricName; } + + @Override + public String getMetricIdentifier(String metricName, CharacterFilter filter,int indexReporter ) { --- End diff -- missing space after second comma. --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request #2517: [FLINK-4564] [metrics] Delimiter should be configu...
Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/2517#discussion_r80239606 --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistry.java --- @@ -99,10 +99,17 @@ public MetricRegistry(Configuration config) { DelegatingConfiguration reporterConfig = new DelegatingConfiguration(config, ConfigConstants.METRICS_REPORTER_PREFIX + namedReporter + "."); final String className = reporterConfig.getString(ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, null); if (className == null) { - LOG.error("No reporter class set for reporter " + namedReporter + ". Metrics might not be exposed/reported."); + LOG.error("No reporter class set for reporter {}. Metrics might not be exposed/reported.",namedReporter); continue; } + String delimiterForReporter = reporterConfig.getString(ConfigConstants.METRICS_REPORTER_SCOPE_DELIMITER,defaultDelimiter); --- End diff -- missing spaces after commas, if and around operands. --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request #2517: [FLINK-4564] [metrics] Delimiter should be configu...
Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/2517#discussion_r80240685 --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistry.java --- @@ -219,9 +235,19 @@ public ScopeFormats getScopeFormats() { public void register(Metric metric, String metricName, MetricGroup group) { try { if (reporters != null) { - for (MetricReporter reporter : reporters) { - if (reporter != null) { - reporter.notifyOfAddedMetric(metric, metricName, group); + for (int i= 0; i
[GitHub] flink pull request #2517: [FLINK-4564] [metrics] Delimiter should be configu...
Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/2517#discussion_r80241072 --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/AbstractMetricGroup.java --- @@ -168,21 +168,40 @@ public String getMetricIdentifier(String metricName) { * @param filter character filter which is applied to the scope components if not null. * @return fully qualified metric name */ - public String getMetricIdentifier(String metricName, CharacterFilter filter) { - if (scopeString == null) { - if (filter != null) { - scopeString = ScopeFormat.concat(filter, registry.getDelimiter(), scopeComponents); - } else { - scopeString = ScopeFormat.concat(registry.getDelimiter(), scopeComponents); - } + public String getMetricIdentifier(String metricName,CharacterFilter filter) { + if (filter != null){ + scopeString = ScopeFormat.concat(filter, registry.getDelimiter(), scopeComponents); + } else { + scopeString = ScopeFormat.concat(registry.getDelimiter(), scopeComponents); } - - if (filter != null) { + if (filter != null){ return scopeString + registry.getDelimiter() + filter.filterCharacters(metricName); } else { return scopeString + registry.getDelimiter() + metricName; } } + + /** +* Returns the fully qualified metric name for reporter with certain number position in group, for example +* {@code "host-7.taskmanager-2.window_word_count.my-mapper.metricName"} +* +* @param metricName metric name +* @param filter character filter which is applied to the scope components if not null. +* @param indexReporter position number of reporter in group reporters +* @return fully qualified metric name +*/ + public String getMetricIdentifier(String metricName, CharacterFilter filter, int indexReporter) { + if (filter != null){ + scopeString = ScopeFormat.concat(filter, registry.getDelimiter(indexReporter), scopeComponents); + } else { + scopeString = ScopeFormat.concat(registry.getDelimiter(indexReporter), scopeComponents); + } + if (filter != null){ --- End diff -- missing space before ```{``` --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request #2517: [FLINK-4564] [metrics] Delimiter should be configu...
Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/2517#discussion_r80240221 --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistry.java --- @@ -163,7 +170,16 @@ public void startQueryService(ActorSystem actorSystem) { } public char getDelimiter() { - return this.delimiter; + return this.globalDelimiter; + } + + public char getDelimiter(int reporterIndex) { --- End diff -- missing javadocs --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request #2517: [FLINK-4564] [metrics] Delimiter should be configu...
Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/2517#discussion_r80239675 --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistry.java --- @@ -99,10 +99,17 @@ public MetricRegistry(Configuration config) { DelegatingConfiguration reporterConfig = new DelegatingConfiguration(config, ConfigConstants.METRICS_REPORTER_PREFIX + namedReporter + "."); final String className = reporterConfig.getString(ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, null); if (className == null) { - LOG.error("No reporter class set for reporter " + namedReporter + ". Metrics might not be exposed/reported."); + LOG.error("No reporter class set for reporter {}. Metrics might not be exposed/reported.",namedReporter); continue; } + String delimiterForReporter = reporterConfig.getString(ConfigConstants.METRICS_REPORTER_SCOPE_DELIMITER,defaultDelimiter); + if(delimiterForReporter.length()!=1){ + LOG.warn("Failed to parse delimiter for reporter {}, using global delimiter.",namedReporter); --- End diff -- let's include the global delimiter in the message as well. --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request #2517: [FLINK-4564] [metrics] Delimiter should be configu...
Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/2517#discussion_r80241187 --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/groups/AbstractMetricGroup.java --- @@ -168,21 +168,40 @@ public String getMetricIdentifier(String metricName) { * @param filter character filter which is applied to the scope components if not null. * @return fully qualified metric name */ - public String getMetricIdentifier(String metricName, CharacterFilter filter) { - if (scopeString == null) { - if (filter != null) { - scopeString = ScopeFormat.concat(filter, registry.getDelimiter(), scopeComponents); - } else { - scopeString = ScopeFormat.concat(registry.getDelimiter(), scopeComponents); - } + public String getMetricIdentifier(String metricName,CharacterFilter filter) { + if (filter != null){ + scopeString = ScopeFormat.concat(filter, registry.getDelimiter(), scopeComponents); + } else { + scopeString = ScopeFormat.concat(registry.getDelimiter(), scopeComponents); } - - if (filter != null) { + if (filter != null){ return scopeString + registry.getDelimiter() + filter.filterCharacters(metricName); } else { return scopeString + registry.getDelimiter() + metricName; } } + + /** +* Returns the fully qualified metric name for reporter with certain number position in group, for example +* {@code "host-7.taskmanager-2.window_word_count.my-mapper.metricName"} +* +* @param metricName metric name +* @param filter character filter which is applied to the scope components if not null. +* @param indexReporter position number of reporter in group reporters --- End diff -- "index of the reporter whose delimiter should be used" --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request #2517: [FLINK-4564] [metrics] Delimiter should be configu...
Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/2517#discussion_r80241834 --- Diff: flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/MetricGroup.java --- @@ -189,4 +189,15 @@ * @return fully qualified metric name */ String getMetricIdentifier(String metricName, CharacterFilter filter); + + /** +* Returns the fully qualified metric name for reporter with certain number position in group, for example +* {@code "host-7.taskmanager-2.window_word_count.my-mapper.metricName"} +* +* @param metricName metric name +* @param filter character filter which is applied to the scope components if not null. +* @param indexReporter position number of reporter in group reporters +* @return fully qualified metric name +*/ + String getMetricIdentifier(String metricName, CharacterFilter filter,int indexReporter); --- End diff -- This method should be part of `AbstractMetricGroup`. The signature of the `MetricRegistry#register()` may be changed if necessary. --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request #2517: [FLINK-4564] [metrics] Delimiter should be configu...
Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/2517#discussion_r80240626 --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistry.java --- @@ -219,9 +235,19 @@ public ScopeFormats getScopeFormats() { public void register(Metric metric, String metricName, MetricGroup group) { try { if (reporters != null) { - for (MetricReporter reporter : reporters) { - if (reporter != null) { - reporter.notifyOfAddedMetric(metric, metricName, group); + for (int i= 0; i
[GitHub] flink pull request #2517: [FLINK-4564] [metrics] Delimiter should be configu...
Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/2517#discussion_r80240523 --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistry.java --- @@ -219,9 +235,19 @@ public ScopeFormats getScopeFormats() { public void register(Metric metric, String metricName, MetricGroup group) { try { if (reporters != null) { - for (MetricReporter reporter : reporters) { - if (reporter != null) { - reporter.notifyOfAddedMetric(metric, metricName, group); + for (int i= 0; i
[GitHub] flink pull request #2517: [FLINK-4564] [metrics] Delimiter should be configu...
Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/2517#discussion_r80239515 --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistry.java --- @@ -99,10 +99,17 @@ public MetricRegistry(Configuration config) { DelegatingConfiguration reporterConfig = new DelegatingConfiguration(config, ConfigConstants.METRICS_REPORTER_PREFIX + namedReporter + "."); final String className = reporterConfig.getString(ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, null); if (className == null) { - LOG.error("No reporter class set for reporter " + namedReporter + ". Metrics might not be exposed/reported."); + LOG.error("No reporter class set for reporter {}. Metrics might not be exposed/reported.",namedReporter); --- End diff -- missing space after comma. --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request #2517: [FLINK-4564] [metrics] Delimiter should be configu...
Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/2517#discussion_r80240342 --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistry.java --- @@ -163,7 +170,16 @@ public void startQueryService(ActorSystem actorSystem) { } public char getDelimiter() { - return this.delimiter; + return this.globalDelimiter; + } + + public char getDelimiter(int reporterIndex) { + try { + return delimiters.get(reporterIndex); + }catch (IndexOutOfBoundsException e){ + LOG.warn("Delimiter for index {} not found return global delimiter",reporterIndex); --- End diff -- missing space after comma. Message should be "Delimiter for index {} not found, returning global delimiter." --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request #2517: [FLINK-4564] [metrics] Delimiter should be configu...
Github user zentol commented on a diff in the pull request: https://github.com/apache/flink/pull/2517#discussion_r80238976 --- Diff: flink-metrics/flink-metrics-core/src/main/java/org/apache/flink/metrics/MetricGroup.java --- @@ -189,4 +189,15 @@ * @return fully qualified metric name */ String getMetricIdentifier(String metricName, CharacterFilter filter); + + /** +* Returns the fully qualified metric name for reporter with certain number position in group, for example +* {@code "host-7.taskmanager-2.window_word_count.my-mapper.metricName"} +* +* @param metricName metric name +* @param filter character filter which is applied to the scope components if not null. +* @param indexReporter position number of reporter in group reporters +* @return fully qualified metric name +*/ + String getMetricIdentifier(String metricName, CharacterFilter filter,int indexReporter); --- End diff -- missing space after second comma. --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request #2517: [FLINK-4564] [metrics] Delimiter should be configu...
GitHub user ex00 reopened a pull request: https://github.com/apache/flink/pull/2517 [FLINK-4564] [metrics] Delimiter should be configured per reporter Hi It is my fix FLINK-4564. I want fix this issue, please send me your comments about this implementation. I could assign to me [issue](https://issues.apache.org/jira/browse/FLINK-4564) in jira? Thanks. You can merge this pull request into a Git repository by running: $ git pull https://github.com/ex00/flink FLINK-4564 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/2517.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 #2517 commit 4dd3bb08eb48430293e4711136fa839fe92aa85d Author: Anton MushinDate: 2016-09-20T10:09:40Z [FLINK-4564] [metrics] Delimiter should be configured per reporter --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request #2517: [FLINK-4564] [metrics] Delimiter should be configu...
Github user ex00 closed the pull request at: https://github.com/apache/flink/pull/2517 --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] flink pull request #2517: [FLINK-4564] [metrics] Delimiter should be configu...
GitHub user ex00 opened a pull request: https://github.com/apache/flink/pull/2517 [FLINK-4564] [metrics] Delimiter should be configured per reporter Hi It is my fix FLINK-4564. I want fix this issue, please send me your comments about this implementation. I could assign to me [issue](https://issues.apache.org/jira/browse/FLINK-4564) in jira? Thanks. You can merge this pull request into a Git repository by running: $ git pull https://github.com/ex00/flink FLINK-4564 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/2517.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 #2517 commit 4dd3bb08eb48430293e4711136fa839fe92aa85d Author: Anton MushinDate: 2016-09-20T10:09:40Z [FLINK-4564] [metrics] Delimiter should be configured per reporter --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---