Yes indeed, it affects the behavior on java 11. I have created a bug in jira about it: Summary: MetricReporter: "metrics.reporters" configuration has to be provided for reporters to be taken into account Key: FLINK-11413 URL: https://issues.apache.org/jira/browse/FLINK-11413 Project: Flink Issue Type: Bug Components: Configuration Affects Versions: 1.7.1
I will have time to fix it and submit a PR. Regards Matthieu Bonneviot Le mer. 23 janv. 2019 à 10:41, Chesnay Schepler <ches...@apache.org> a écrit : > nvm, it does indeed affect behavior :/ > > On 23.01.2019 10:08, Chesnay Schepler wrote: > > Just to make sure, this issue does not actually affect the behavior, > > does it? Since we only use these as a filter for reporters to activate. > > > > On 21.01.2019 18:22, Matthieu Bonneviot wrote: > >> Hi > >> > >> I don't have the jira permission but If you grant me the permission I > >> could > >> contribute to fix the following issue: > >> When using java 11, "metrics.reporters" configuration has to be provided > >> for reporters to be taken into account. > >> > >> The desired behavior: > >> The MetricRegistryConfiguration looks for a conf like > >> "metrics.reporters = > >> foo,bar", if not found: all reporters that could be found in the > >> configuration will be started. > >> > >> In the code is it done by > >> Set<String> includedReporters = > >> > reporterListPattern.splitAsStream(includedReportersString).collect(Collectors.toSet()); > > >> > >> > https://github.com/apache/flink/blob/master/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistryConfiguration.java#L134 > >> > >> > >> Definition of splitAsStream: If this pattern does not match any > >> subsequence > >> of the input then the resulting stream has just one element, namely the > >> input sequence in string form. > >> It means reporterListPattern.splitAsStream("") should return "" and so > >> includedReporters should have size 1 with "" as unique element > >> > >> However there is a misbehavior in some version of java 8, it does return > >> empty stream. > >> But working with java 11, the further code does not work: if > >> (includedReporters.isEmpty() || > >> includedReporters.contains(reporterName)) > >> > https://github.com/apache/flink/blob/master/flink-runtime/src/main/java/org/apache/flink/runtime/metrics/MetricRegistryConfiguration.java#L145 > >> > >> > >> I would suggest to filter empty string: > >> Set<String> includedReporters = > >> reporterListPattern.splitAsStream(includedReportersString).*filter(s -> > >> !s.isEmpty())*.collect(Collectors.toSet()); > >> > >> Regards > >> Matthieu Bonneviot > > > > > > > > -- Matthieu Bonneviot Senior Engineer, DataDome M +33 7 68 29 79 34 <+33+7+68+29+79+34> E matthieu.bonnev...@datadome.co <matthieu.bonnev...@datadome.co> W www.datadome.co <http://www.datadome.co?utm_source=WiseStamp&utm_medium=email&utm_term=&utm_content=&utm_campaign=signature> <https://www.facebook.com/datadome/?utm_source=WiseStamp&utm_medium=email&utm_term=&utm_content=&utm_campaign=signature> <https://fr.linkedin.com/company/datadome?utm_source=WiseStamp&utm_medium=email&utm_term=&utm_content=&utm_campaign=signature> <https://twitter.com/data_dome?utm_source=WiseStamp&utm_medium=email&utm_term=&utm_content=&utm_campaign=signature> <https://datadome.co/forrester-strong-performer-2018/?utm_source=WiseStamp&utm_medium=email&utm_term=&utm_content=&utm_campaign=signature> DataDome ranked 'Strong Performer' in latest Forrester Bot management report <https://datadome.co/forrester-strong-performer-2018/?utm_source=WiseStamp&utm_medium=email&utm_term=&utm_content=&utm_campaign=signature>